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

kudu git commit: rpc: use TLS only for authentication for loopback connections

Repository: kudu
Updated Branches:
  refs/heads/master 61f95748b -> 49c48174d


rpc: use TLS only for authentication for loopback connections

This adds a new feature flag TLS_AUTHENTICATION_ONLY which is advertised
by the client and server in the case that they detect that the remote
peer is on the same socket address as the local one. When both peers
advertise this flag, TLS authentication proceeds as usual, but after a
successful handshake the connection is _not_ wrapped with TLS.

In case any users take fault with this optimization, an escape hatch
flag is provided which can disable it.

This patch only implements this feature on the C++ side. The Java side
will be done in a future patch.

Change-Id: I5bb637b3b06de4c4571bf2411aa0aaf8e1f41c2f
Reviewed-on: http://gerrit.cloudera.org:8080/5976
Tested-by: Kudu Jenkins
Reviewed-by: Alexey Serbin <as...@cloudera.com>


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

Branch: refs/heads/master
Commit: 49c48174d17ae74925bebab0be3ef2a02a0d786b
Parents: 61f9574
Author: Todd Lipcon <to...@apache.org>
Authored: Sun Feb 12 20:20:22 2017 -0800
Committer: Todd Lipcon <to...@apache.org>
Committed: Mon Feb 13 22:09:37 2017 +0000

----------------------------------------------------------------------
 docs/design-docs/rpc.md                         | 24 ++++++++++-
 .../java/org/apache/kudu/client/Negotiator.java |  2 +
 src/kudu/rpc/client_negotiation.cc              | 20 ++++++++--
 src/kudu/rpc/client_negotiation.h               | 13 +++++-
 src/kudu/rpc/constants.cc                       |  3 ++
 src/kudu/rpc/negotiation-test.cc                | 17 ++++++--
 src/kudu/rpc/negotiation.cc                     |  8 ++++
 src/kudu/rpc/rpc_header.proto                   |  9 +++++
 src/kudu/rpc/server_negotiation.cc              | 25 +++++++++---
 src/kudu/rpc/server_negotiation.h               |  9 ++++-
 src/kudu/security/tls_handshake.cc              | 42 +++++++++++++++++++-
 src/kudu/security/tls_handshake.h               | 23 +++++++++++
 src/kudu/security/tls_socket.cc                 | 24 -----------
 src/kudu/security/tls_socket.h                  | 10 -----
 src/kudu/util/net/socket.cc                     | 11 +++++
 src/kudu/util/net/socket.h                      |  6 +++
 16 files changed, 194 insertions(+), 52 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/49c48174/docs/design-docs/rpc.md
----------------------------------------------------------------------
diff --git a/docs/design-docs/rpc.md b/docs/design-docs/rpc.md
index 23d697a..cf256e0 100644
--- a/docs/design-docs/rpc.md
+++ b/docs/design-docs/rpc.md
@@ -443,8 +443,28 @@ Client                                                                    Server
 ```
 
 The client and server repeat `TLS_HANDSHAKE` round-trips until the TLS handshake
-is complete, at which point both ends wrap their respective TCP socket in the
-new TLS channel. All subsequent messages will be encrypted.
+is complete. After the handshake is complete, in the absence of the
+`TLS_AUTHENTICATION_ONLY` feature described belowq, the client and server
+wrap the socket connection such that all further traffic is encrypted via TLS.
+
+##### `TLS_AUTHENTICATION_ONLY`
+
+In many cases, RPC connections are made between a client and server running
+on the same machine. For example, the schedulers in Spark, MapReduce,
+or Impala will typically schedule tasks such that most data is scanned by
+a process running on the same machine as the Kudu replica. In this case,
+the overhead of TLS is significant, and TLS affords no additional security
+(only a local root attacker could intercept the traffic, and such an
+attacker could just as easily dump the process memory or read data from disk).
+
+To avoid TLS overhead in such situations, while keeping the authentication
+properties of TLS, KRPC supports the `TLS_AUTHENTICATION_ONLY` feature flag.
+In the case that a client or server detects that the remote peer is on the
+same socket address as the local peer, it will advertise this flag. If both
+client and server advertise the flag, then after completing the TLS handshake,
+the peers will _not_ wrap the socket with TLS. All other behavior remains
+the same, including channel binding as described below.
+
 
 #### Step 3: SASL Negotiation
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/49c48174/java/kudu-client/src/main/java/org/apache/kudu/client/Negotiator.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/main/java/org/apache/kudu/client/Negotiator.java b/java/kudu-client/src/main/java/org/apache/kudu/client/Negotiator.java
index eba71a3..cc43832 100644
--- a/java/kudu-client/src/main/java/org/apache/kudu/client/Negotiator.java
+++ b/java/kudu-client/src/main/java/org/apache/kudu/client/Negotiator.java
@@ -158,6 +158,8 @@ public class Negotiator extends SimpleChannelUpstreamHandler {
     for (RpcHeader.RpcFeatureFlag flag : SUPPORTED_RPC_FEATURES) {
       builder.addSupportedFeatures(flag);
     }
+    // TODO(todd): if we are on a loopback connection, advertise and support
+    // TLS_AUTHENTICATION_ONLY.
 
     builder.setStep(RpcHeader.NegotiatePB.NegotiateStep.NEGOTIATE);
     state = State.AWAIT_NEGOTIATE;

http://git-wip-us.apache.org/repos/asf/kudu/blob/49c48174/src/kudu/rpc/client_negotiation.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/client_negotiation.cc b/src/kudu/rpc/client_negotiation.cc
index 45bdc25..9c48332 100644
--- a/src/kudu/rpc/client_negotiation.cc
+++ b/src/kudu/rpc/client_negotiation.cc
@@ -25,6 +25,7 @@
 #include <set>
 #include <string>
 
+#include <gflags/gflags.h>
 #include <glog/logging.h>
 #include <sasl/sasl.h>
 
@@ -58,6 +59,8 @@ using std::unique_ptr;
 
 using strings::Substitute;
 
+DECLARE_bool(rpc_encrypt_loopback_connections);
+
 namespace kudu {
 namespace rpc {
 
@@ -291,7 +294,14 @@ Status ClientNegotiation::SendNegotiate() {
   msg.set_step(NegotiatePB::NEGOTIATE);
 
   // Advertise our supported features.
-  for (RpcFeatureFlag feature : kSupportedClientRpcFeatureFlags) {
+  client_features_ = kSupportedClientRpcFeatureFlags;
+  // If the remote peer is local, then we allow using TLS for authentication
+  // without encryption or integrity.
+  if (socket_->IsLoopbackConnection() && !FLAGS_rpc_encrypt_loopback_connections) {
+    client_features_.insert(TLS_AUTHENTICATION_ONLY);
+  }
+
+  for (RpcFeatureFlag feature : client_features_) {
     msg.add_supported_features(feature);
   }
 
@@ -400,6 +410,11 @@ Status ClientNegotiation::HandleTlsHandshake(const NegotiatePB& response) {
 
   // TLS handshake is finished.
   DCHECK(token.empty());
+
+  if (ContainsKey(server_features_, TLS_AUTHENTICATION_ONLY) &&
+      ContainsKey(client_features_, TLS_AUTHENTICATION_ONLY)) {
+    return tls_handshake_.FinishNoWrap(*socket_);
+  }
   return tls_handshake_.Finish(&socket_);
 }
 
@@ -490,9 +505,8 @@ Status ClientNegotiation::HandleSaslSuccess(const NegotiatePB& response) {
   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));
+    RETURN_NOT_OK(tls_handshake_.GetRemoteCert(&cert));
 
     string expected_channel_bindings;
     RETURN_NOT_OK_PREPEND(cert.GetServerEndPointChannelBindings(&expected_channel_bindings),

http://git-wip-us.apache.org/repos/asf/kudu/blob/49c48174/src/kudu/rpc/client_negotiation.h
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/client_negotiation.h b/src/kudu/rpc/client_negotiation.h
index 547eeee..b33694e 100644
--- a/src/kudu/rpc/client_negotiation.h
+++ b/src/kudu/rpc/client_negotiation.h
@@ -68,9 +68,15 @@ class ClientNegotiation {
   Status EnableGSSAPI();
 
   // Returns mechanism negotiated by this connection.
-  // Must be called before Negotiate().
+  // Must be called after Negotiate().
   SaslMechanism::Type negotiated_mechanism() const;
 
+  // Returns true if TLS was negotiated.
+  // Must be called after Negotiate().
+  bool tls_negotiated() const {
+    return tls_negotiated_;
+  }
+
   // Returns the set of RPC system features supported by the remote server.
   // Must be called before Negotiate().
   std::set<RpcFeatureFlag> server_features() const {
@@ -189,6 +195,11 @@ class ClientNegotiation {
   std::string plain_pass_;
   gscoped_ptr<sasl_secret_t, FreeDeleter> psecret_;
 
+  // The set of features advertised by the client. Filled in when we send
+  // the first message. This is not necessarily constant since some features
+  // may be dynamically enabled.
+  std::set<RpcFeatureFlag> client_features_;
+
   // The set of features supported by the server. Filled in during negotiation.
   std::set<RpcFeatureFlag> server_features_;
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/49c48174/src/kudu/rpc/constants.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/constants.cc b/src/kudu/rpc/constants.cc
index 7f91a51..c6a93ce 100644
--- a/src/kudu/rpc/constants.cc
+++ b/src/kudu/rpc/constants.cc
@@ -28,6 +28,9 @@ const char* const kSaslProtoName = "kudu";
 
 // The server supports the TLS flag if there is a TLS certificate available.
 // The flag is added during negotiation if this is the case.
+//
+// NOTE: the TLS_AUTHENTICATION_ONLY flag is dynamically added on both sides
+// based on the remote peer's address.
 set<RpcFeatureFlag> kSupportedServerRpcFeatureFlags = { APPLICATION_FEATURE_FLAGS };
 set<RpcFeatureFlag> kSupportedClientRpcFeatureFlags = { APPLICATION_FEATURE_FLAGS, TLS };
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/49c48174/src/kudu/rpc/negotiation-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/negotiation-test.cc b/src/kudu/rpc/negotiation-test.cc
index 37aff51..9cd4f97 100644
--- a/src/kudu/rpc/negotiation-test.cc
+++ b/src/kudu/rpc/negotiation-test.cc
@@ -37,6 +37,7 @@
 #include "kudu/rpc/server_negotiation.h"
 #include "kudu/security/test/mini_kdc.h"
 #include "kudu/security/tls_context.h"
+#include "kudu/security/tls_socket.h"
 #include "kudu/util/monotime.h"
 #include "kudu/util/net/sockaddr.h"
 #include "kudu/util/net/socket.h"
@@ -60,6 +61,7 @@ using std::unique_ptr;
 DEFINE_bool(is_test_child, false,
             "Used by tests which require clean processes. "
             "See TestDisableInit.");
+DECLARE_bool(rpc_encrypt_loopback_connections);
 
 namespace kudu {
 namespace rpc {
@@ -583,7 +585,9 @@ static void RunTlsGssapiNegotiationServer(unique_ptr<Socket> socket) {
   ASSERT_EQ(SaslMechanism::GSSAPI, server_negotiation.negotiated_mechanism());
   ASSERT_EQ("testuser", server_negotiation.authenticated_user());
 
-  // TODO(PKI): assert that TLS is actually negotiated.
+  CHECK(server_negotiation.tls_negotiated());
+  bool is_tls_socket = dynamic_cast<security::TlsSocket*>(server_negotiation.socket());
+  CHECK_EQ(is_tls_socket, FLAGS_rpc_encrypt_loopback_connections);
 }
 
 static void RunTlsGssapiNegotiationClient(unique_ptr<Socket> socket) {
@@ -597,7 +601,9 @@ static void RunTlsGssapiNegotiationClient(unique_ptr<Socket> socket) {
   CHECK(ContainsKey(client_negotiation.server_features(), APPLICATION_FEATURE_FLAGS));
   CHECK(ContainsKey(client_negotiation.server_features(), TLS));
 
-  // TODO(PKI): assert that TLS is actually negotiated.
+  CHECK(client_negotiation.tls_negotiated());
+  bool is_tls_socket = dynamic_cast<security::TlsSocket*>(client_negotiation.socket());
+  CHECK_EQ(is_tls_socket, FLAGS_rpc_encrypt_loopback_connections);
 }
 
 // Test TLS and GSSAPI (kerberos) SASL negotiation.
@@ -616,7 +622,12 @@ TEST_F(TestNegotiation, TestTlsWithGssapiNegotiation) {
   ASSERT_OK(kdc.SetKrb5Environment());
 
   // Authentication should succeed on both sides.
-  RunNegotiationTest(RunTlsGssapiNegotiationServer, RunTlsGssapiNegotiationClient);
+  // We run with both flag values for the loopback TLS encryption flag.
+  for (bool encrypt_loopback : { false, true }) {
+    SCOPED_TRACE(encrypt_loopback);
+    FLAGS_rpc_encrypt_loopback_connections = encrypt_loopback;
+    RunNegotiationTest(RunTlsGssapiNegotiationServer, RunTlsGssapiNegotiationClient);
+  }
 }
 
 ////////////////////////////////////////////////////////////////////////////////

http://git-wip-us.apache.org/repos/asf/kudu/blob/49c48174/src/kudu/rpc/negotiation.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/negotiation.cc b/src/kudu/rpc/negotiation.cc
index 4e86292..c31b1f8 100644
--- a/src/kudu/rpc/negotiation.cc
+++ b/src/kudu/rpc/negotiation.cc
@@ -54,6 +54,14 @@ TAG_FLAG(rpc_negotiation_inject_delay_ms, unsafe);
 
 DECLARE_bool(server_require_kerberos);
 
+DEFINE_bool(rpc_encrypt_loopback_connections, false,
+            "Whether to encrypt data transfer on RPC connections that stay within "
+            "a single host. Encryption here is likely to offer no additional "
+            "security benefit since only a local 'root' user could intercept the "
+            "traffic, and wire encryption does not suitably protect against such "
+            "an attacker.");
+TAG_FLAG(rpc_encrypt_loopback_connections, advanced);
+
 using strings::Substitute;
 
 namespace kudu {

http://git-wip-us.apache.org/repos/asf/kudu/blob/49c48174/src/kudu/rpc/rpc_header.proto
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/rpc_header.proto b/src/kudu/rpc/rpc_header.proto
index 3159688..5d426e3 100644
--- a/src/kudu/rpc/rpc_header.proto
+++ b/src/kudu/rpc/rpc_header.proto
@@ -74,6 +74,15 @@ enum RpcFeatureFlag {
   // this flag, the connection will automatically be wrapped in a TLS protected
   // channel following a TLS handshake.
   TLS = 2;
+
+  // If both sides advertise TLS_AUTHENTICATION_ONLY, this means that they
+  // agree that, after handshaking TLS, they will *not* wrap the connection
+  // in a TLS-protected channel. Instead, they will use TLS only for its
+  // handshake-based authentication.
+  //
+  // This is currently used for loopback connections only, so that compute
+  // frameworks which schedule for locality don't pay encryption overhead.
+  TLS_AUTHENTICATION_ONLY = 3;
 };
 
 // Message type passed back & forth for the SASL negotiation.

http://git-wip-us.apache.org/repos/asf/kudu/blob/49c48174/src/kudu/rpc/server_negotiation.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/server_negotiation.cc b/src/kudu/rpc/server_negotiation.cc
index 77cae47..00c66e8 100644
--- a/src/kudu/rpc/server_negotiation.cc
+++ b/src/kudu/rpc/server_negotiation.cc
@@ -23,6 +23,7 @@
 #include <set>
 #include <string>
 
+#include <gflags/gflags.h>
 #include <glog/logging.h>
 #include <google/protobuf/message_lite.h>
 #include <sasl/sasl.h>
@@ -47,6 +48,8 @@ using std::set;
 using std::string;
 using std::unique_ptr;
 
+DECLARE_bool(rpc_encrypt_loopback_connections);
+
 namespace kudu {
 namespace rpc {
 
@@ -356,11 +359,18 @@ Status ServerNegotiation::SendNegotiate(const set<SaslMechanism::Type>& server_m
   }
 
   // Tell the client which features we support.
-  for (RpcFeatureFlag feature : kSupportedServerRpcFeatureFlags) {
-    response.add_supported_features(feature);
-  }
+  server_features_ = kSupportedServerRpcFeatureFlags;
   if (tls_context_->has_cert()) {
-    response.add_supported_features(TLS);
+    server_features_.insert(TLS);
+    // If the remote peer is local, then we allow using TLS for authentication
+    // without encryption or integrity.
+    if (socket_->IsLoopbackConnection() && !FLAGS_rpc_encrypt_loopback_connections) {
+      server_features_.insert(TLS_AUTHENTICATION_ONLY);
+    }
+  }
+
+  for (RpcFeatureFlag feature : server_features_) {
+    response.add_supported_features(feature);
   }
 
   RETURN_NOT_OK(SendNegotiatePB(response));
@@ -396,6 +406,10 @@ Status ServerNegotiation::HandleTlsHandshake(const NegotiatePB& request) {
   RETURN_NOT_OK(s);
 
   // TLS handshake is finished.
+  if (ContainsKey(server_features_, TLS_AUTHENTICATION_ONLY) &&
+      ContainsKey(client_features_, TLS_AUTHENTICATION_ONLY)) {
+    return tls_handshake_.FinishNoWrap(*socket_);
+  }
   return tls_handshake_.Finish(&socket_);
 }
 
@@ -537,9 +551,8 @@ Status ServerNegotiation::SendSaslSuccess() {
 
   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));
+    RETURN_NOT_OK(tls_handshake_.GetLocalCert(&cert));
 
     string plaintext_channel_bindings;
     RETURN_NOT_OK(cert.GetServerEndPointChannelBindings(&plaintext_channel_bindings));

http://git-wip-us.apache.org/repos/asf/kudu/blob/49c48174/src/kudu/rpc/server_negotiation.h
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/server_negotiation.h b/src/kudu/rpc/server_negotiation.h
index 05eb731..ee5d395 100644
--- a/src/kudu/rpc/server_negotiation.h
+++ b/src/kudu/rpc/server_negotiation.h
@@ -70,6 +70,12 @@ class ServerNegotiation {
   // Must be called after Negotiate().
   SaslMechanism::Type negotiated_mechanism() const;
 
+  // Returns true if TLS was negotiated.
+  // Must be called after Negotiate().
+  bool tls_negotiated() const {
+    return tls_negotiated_;
+  }
+
   // Returns the set of RPC system features supported by the remote client.
   // Must be called after Negotiate().
   std::set<RpcFeatureFlag> client_features() const {
@@ -185,8 +191,9 @@ class ServerNegotiation {
   security::TlsHandshake tls_handshake_;
   bool tls_negotiated_;
 
-  // The set of features supported by the client. Filled in during negotiation.
+  // The set of features supported by the client and server. Filled in during negotiation.
   std::set<RpcFeatureFlag> client_features_;
+  std::set<RpcFeatureFlag> server_features_;
 
   // The successfully-authenticated user, if applicable. Filled in during
   // negotiation.

http://git-wip-us.apache.org/repos/asf/kudu/blob/49c48174/src/kudu/security/tls_handshake.cc
----------------------------------------------------------------------
diff --git a/src/kudu/security/tls_handshake.cc b/src/kudu/security/tls_handshake.cc
index 522fee6..123a123 100644
--- a/src/kudu/security/tls_handshake.cc
+++ b/src/kudu/security/tls_handshake.cc
@@ -25,6 +25,7 @@
 #include <openssl/x509.h>
 #include <openssl/x509v3.h>
 
+#include "kudu/security/cert.h"
 #include "kudu/security/tls_socket.h"
 #include "kudu/util/net/sockaddr.h"
 #include "kudu/util/status.h"
@@ -130,7 +131,7 @@ Status TlsHandshake::Verify(const Socket& socket) const {
   }
 
   // Get the peer certificate.
-  c_unique_ptr<X509> cert = ssl_make_unique(SSL_get_peer_certificate(ssl_.get()));
+  X509* cert = remote_cert_.GetRawData();
   if (!cert) {
     if (SSL_get_verify_mode(ssl_.get()) & SSL_VERIFY_FAIL_IF_NO_PEER_CERT) {
       return Status::NotAuthorized("Handshake failed: unable to retreive peer certificate");
@@ -154,7 +155,7 @@ Status TlsHandshake::Verify(const Socket& socket) const {
 
   // Check if the hostname matches with either the Common Name or any of the Subject Alternative
   // Names of the certificate.
-  int match = X509_check_host(cert.get(),
+  int match = X509_check_host(cert,
                               peer_hostname.c_str(),
                               peer_hostname.length(),
                               0,
@@ -169,7 +170,23 @@ Status TlsHandshake::Verify(const Socket& socket) const {
   return Status::OK();
 }
 
+Status TlsHandshake::GetCerts() {
+  X509* cert = SSL_get_certificate(ssl_.get());
+  if (cert) {
+    // For whatever reason, SSL_get_certificate (unlike SSL_get_peer_certificate)
+    // does not increment the X509's reference count.
+    local_cert_.AdoptAndAddRefRawData(cert);
+  }
+
+  cert = SSL_get_peer_certificate(ssl_.get());
+  if (cert) {
+    remote_cert_.AdoptRawData(cert);
+  }
+  return Status::OK();
+}
+
 Status TlsHandshake::Finish(unique_ptr<Socket>* socket) {
+  RETURN_NOT_OK(GetCerts());
   RETURN_NOT_OK(Verify(**socket));
 
   int fd = (*socket)->Release();
@@ -187,5 +204,26 @@ Status TlsHandshake::Finish(unique_ptr<Socket>* socket) {
   return Status::OK();
 }
 
+Status TlsHandshake::FinishNoWrap(const Socket& socket) {
+  RETURN_NOT_OK(GetCerts());
+  return Verify(socket);
+}
+
+Status TlsHandshake::GetLocalCert(Cert* cert) const {
+  if (!local_cert_.GetRawData()) {
+    return Status::RuntimeError("no local certificate");
+  }
+  cert->AdoptAndAddRefRawData(local_cert_.GetRawData());
+  return Status::OK();
+}
+
+Status TlsHandshake::GetRemoteCert(Cert* cert) const {
+  if (!remote_cert_.GetRawData()) {
+    return Status::RuntimeError("no remote certificate");
+  }
+  cert->AdoptAndAddRefRawData(remote_cert_.GetRawData());
+  return Status::OK();
+}
+
 } // namespace security
 } // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/49c48174/src/kudu/security/tls_handshake.h
----------------------------------------------------------------------
diff --git a/src/kudu/security/tls_handshake.h b/src/kudu/security/tls_handshake.h
index 4937578..1b293d5 100644
--- a/src/kudu/security/tls_handshake.h
+++ b/src/kudu/security/tls_handshake.h
@@ -22,6 +22,7 @@
 
 #include <glog/logging.h>
 
+#include "kudu/security/cert.h"
 #include "kudu/security/openssl_util.h"
 #include "kudu/util/net/socket.h"
 #include "kudu/util/status.h"
@@ -104,6 +105,22 @@ class TlsHandshake {
   // calling this.
   Status Finish(std::unique_ptr<Socket>* socket);
 
+  // Finish the handshake, using the provided socket to verify the remote peer,
+  // but without wrapping the socket.
+  Status FinishNoWrap(const Socket& socket);
+
+  // Retrieve the local certificate. This will return an error status if there
+  // is no local certificate.
+  //
+  // May only be called after 'Finish' or 'FinishNoWrap'.
+  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.
+  //
+  // May only be called after 'Finish' or 'FinishNoWrap'.
+  Status GetRemoteCert(Cert* cert) const WARN_UNUSED_RESULT;
+
  private:
   friend class TlsContext;
 
@@ -124,11 +141,17 @@ class TlsHandshake {
     return ssl_.get();
   }
 
+  // Populates local_cert_ and remote_cert_.
+  Status GetCerts();
+
   // Verifies that the handshake is valid for the provided socket.
   Status Verify(const Socket& socket) const;
 
   // Owned SSL handle.
   c_unique_ptr<SSL> ssl_;
+
+  Cert local_cert_;
+  Cert remote_cert_;
 };
 
 } // namespace security

http://git-wip-us.apache.org/repos/asf/kudu/blob/49c48174/src/kudu/security/tls_socket.cc
----------------------------------------------------------------------
diff --git a/src/kudu/security/tls_socket.cc b/src/kudu/security/tls_socket.cc
index 27a6af6..b510906 100644
--- a/src/kudu/security/tls_socket.cc
+++ b/src/kudu/security/tls_socket.cc
@@ -37,30 +37,6 @@ 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/49c48174/src/kudu/security/tls_socket.h
----------------------------------------------------------------------
diff --git a/src/kudu/security/tls_socket.h b/src/kudu/security/tls_socket.h
index 1d0b26f..dec55ec 100644
--- a/src/kudu/security/tls_socket.h
+++ b/src/kudu/security/tls_socket.h
@@ -27,21 +27,11 @@ typedef ssl_st SSL;
 namespace kudu {
 namespace security {
 
-class Cert;
-
 class TlsSocket : public Socket {
  public:
 
   ~TlsSocket() 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,

http://git-wip-us.apache.org/repos/asf/kudu/blob/49c48174/src/kudu/util/net/socket.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/net/socket.cc b/src/kudu/util/net/socket.cc
index 82b6601..85e858a 100644
--- a/src/kudu/util/net/socket.cc
+++ b/src/kudu/util/net/socket.cc
@@ -296,6 +296,17 @@ Status Socket::GetPeerAddress(Sockaddr *cur_addr) const {
   return Status::OK();
 }
 
+bool Socket::IsLoopbackConnection() const {
+  Sockaddr local, remote;
+  if (!GetSocketAddress(&local).ok()) return false;
+  if (!GetSocketAddress(&remote).ok()) return false;
+
+  // Compare without comparing ports.
+  local.set_port(0);
+  remote.set_port(0);
+  return local == remote;
+}
+
 Status Socket::Bind(const Sockaddr& bind_addr) {
   struct sockaddr_in addr = bind_addr.addr();
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/49c48174/src/kudu/util/net/socket.h
----------------------------------------------------------------------
diff --git a/src/kudu/util/net/socket.h b/src/kudu/util/net/socket.h
index b3d7eea..1362e67 100644
--- a/src/kudu/util/net/socket.h
+++ b/src/kudu/util/net/socket.h
@@ -99,6 +99,12 @@ class Socket {
   // Call getpeername to get the address of the connected peer.
   Status GetPeerAddress(Sockaddr *cur_addr) const;
 
+  // Return true if this socket is determined to be a loopback connection
+  // (i.e. the local and remote peer share an IP address).
+  //
+  // If any error occurs while determining this, returns false.
+  bool IsLoopbackConnection() const;
+
   // Call bind() to bind the socket to a given address.
   // If bind() fails and indicates that the requested port is already in use,
   // generates an informative log message by calling 'lsof' if available.