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/01 18:40:37 UTC

[2/2] kudu git commit: tls: move setting of verification modes into TlsHandshake

tls: move setting of verification modes into TlsHandshake

Depending on the particular type of connection, we need to configure the
TLS verification differently. For example, a connection that uses a
token for client authentication needs to verify the server cert for
server authentication, whereas the initial connection uses Kerberos for
mutual authentication and doesn't need TLS authentication at all.

As such, the global configuration of SSL verification mode in TlsContext
is no longer appropriate. This moves the configuration to be
per-TlsHandshake.

Change-Id: I37cbc20b1a70a7469a2a2f43702599b1b55ff294
Reviewed-on: http://gerrit.cloudera.org:8080/5839
Reviewed-by: Dan Burkert <da...@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/63ce3ec6
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/63ce3ec6
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/63ce3ec6

Branch: refs/heads/master
Commit: 63ce3ec632c92a3d215cf49ace29a73d51023401
Parents: 8cf3920
Author: Todd Lipcon <to...@apache.org>
Authored: Tue Jan 31 13:29:54 2017 -0800
Committer: Todd Lipcon <to...@apache.org>
Committed: Wed Feb 1 18:39:36 2017 +0000

----------------------------------------------------------------------
 src/kudu/security/tls_context.cc        |   6 +-
 src/kudu/security/tls_handshake-test.cc | 146 ++++++++++++++++++++++++---
 src/kudu/security/tls_handshake.cc      |  60 ++++++++++-
 src/kudu/security/tls_handshake.h       |  38 ++++++-
 4 files changed, 227 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/63ce3ec6/src/kudu/security/tls_context.cc
----------------------------------------------------------------------
diff --git a/src/kudu/security/tls_context.cc b/src/kudu/security/tls_context.cc
index be8e3b6..2eec883 100644
--- a/src/kudu/security/tls_context.cc
+++ b/src/kudu/security/tls_context.cc
@@ -68,8 +68,10 @@ Status TlsContext::Init() {
   SSL_CTX_set_options(ctx_.get(),
                       SSL_OP_NO_SSLv2 | SSL_OP_NO_SSLv3 |
                       SSL_OP_NO_COMPRESSION);
-  SSL_CTX_set_verify(ctx_.get(),
-      SSL_VERIFY_PEER | SSL_VERIFY_FAIL_IF_NO_PEER_CERT | SSL_VERIFY_CLIENT_ONCE, nullptr);
+
+  // TODO(PKI): is it possible to disable client-side renegotiation? it seems there
+  // have been various CVEs related to this feature that we don't need.
+  // TODO(PKI): set desired cipher suites?
   return Status::OK();
 }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/63ce3ec6/src/kudu/security/tls_handshake-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/security/tls_handshake-test.cc b/src/kudu/security/tls_handshake-test.cc
index a8f5f6d..5ee550a 100644
--- a/src/kudu/security/tls_handshake-test.cc
+++ b/src/kudu/security/tls_handshake-test.cc
@@ -17,6 +17,7 @@
 
 #include "kudu/security/tls_handshake.h"
 
+#include <functional>
 #include <string>
 
 #include <gtest/gtest.h>
@@ -30,25 +31,81 @@ using std::string;
 namespace kudu {
 namespace security {
 
-class TestTlsHandshake : public KuduTest {};
-
-TEST_F(TestTlsHandshake, TestHandshake) {
-  string cert_path = GetTestPath("cert.pem");
-  string key_path = GetTestPath("key.pem");
-  ASSERT_OK(CreateSSLServerCert(cert_path));
-  ASSERT_OK(CreateSSLPrivateKey(key_path));
-
-  TlsContext tls_context;
-  ASSERT_OK(tls_context.Init());
-  ASSERT_OK(tls_context.LoadCertificate(cert_path));
-  ASSERT_OK(tls_context.LoadPrivateKey(key_path));
-  ASSERT_OK(tls_context.LoadCertificateAuthority(cert_path));
+class TestTlsHandshake : public KuduTest {
+ public:
+  void SetUp() override {
+    KuduTest::SetUp();
+
+    key_path_ = GetTestPath("key.pem");
+    cert_path_ = GetTestPath("cert.pem");
+    ASSERT_OK(CreateSSLServerCert(cert_path_));
+    ASSERT_OK(CreateSSLPrivateKey(key_path_));
+
+    ASSERT_OK(client_tls_.Init());
+    ASSERT_OK(server_tls_.Init());
+  }
+
+ protected:
+  // Run a handshake using 'client_tls_' and 'server_tls_'. The client and server
+  // verification modes are set to 'client_verify' and 'server_verify' respectively.
+  Status RunHandshake(TlsVerificationMode client_verify,
+                      TlsVerificationMode server_verify) {
+    TlsHandshake client, server;
+    client_tls_.InitiateHandshake(TlsHandshakeType::CLIENT, &client);
+    server_tls_.InitiateHandshake(TlsHandshakeType::SERVER, &server);
+
+    client.set_verification_mode(client_verify);
+    server.set_verification_mode(server_verify);
+
+    bool client_done = false, server_done = false;
+    string to_client;
+    string to_server;
+    while (!client_done || !server_done) {
+      if (!client_done) {
+        Status s = client.Continue(to_client, &to_server);
+        VLOG(1) << "client->server: " << to_server.size() << " bytes";
+        if (s.ok()) {
+          client_done = true;
+        } else if (!s.IsIncomplete()) {
+          CHECK(s.IsRuntimeError());
+          return s.CloneAndPrepend("client error");
+        }
+      }
+      if (!server_done) {
+        CHECK(!client_done);
+        Status s = server.Continue(to_server, &to_client);
+        VLOG(1) << "server->client: " << to_client.size() << " bytes";
+        if (s.ok()) {
+          server_done = true;
+        } else if (!s.IsIncomplete()) {
+          CHECK(s.IsRuntimeError());
+          return s.CloneAndPrepend("server error");
+        }
+      }
+    }
+    return Status::OK();
+  }
+
+  TlsContext client_tls_;
+  TlsContext server_tls_;
+
+  string cert_path_;
+  string key_path_;
+};
+
+TEST_F(TestTlsHandshake, TestSuccessfulHandshake) {
+  // Both client and server have certs and CA.
+  ASSERT_OK(client_tls_.LoadCertificate(cert_path_));
+  ASSERT_OK(client_tls_.LoadPrivateKey(key_path_));
+  ASSERT_OK(client_tls_.LoadCertificateAuthority(cert_path_));
+  ASSERT_OK(server_tls_.LoadCertificate(cert_path_));
+  ASSERT_OK(server_tls_.LoadPrivateKey(key_path_));
+  ASSERT_OK(server_tls_.LoadCertificateAuthority(cert_path_));
 
   TlsHandshake server;
   TlsHandshake client;
-
-  tls_context.InitiateHandshake(TlsHandshakeType::SERVER, &server);
-  tls_context.InitiateHandshake(TlsHandshakeType::CLIENT, &client);
+  client_tls_.InitiateHandshake(TlsHandshakeType::SERVER, &server);
+  server_tls_.InitiateHandshake(TlsHandshakeType::CLIENT, &client);
 
   string buf1;
   string buf2;
@@ -74,5 +131,62 @@ TEST_F(TestTlsHandshake, TestHandshake) {
   ASSERT_EQ(buf2.size(), 0);
 }
 
+// Client has no cert.
+// Server has self-signed cert.
+TEST_F(TestTlsHandshake, Test_ClientNone_ServerSelfSigned) {
+  ASSERT_OK(server_tls_.LoadCertificate(cert_path_));
+  ASSERT_OK(server_tls_.LoadPrivateKey(key_path_));
+
+  // If the client wants to verify the server, it should fail because
+  // the server cert is self-signed.
+  Status s = RunHandshake(TlsVerificationMode::VERIFY_REMOTE_CERT_AND_HOST,
+                          TlsVerificationMode::VERIFY_NONE);
+  ASSERT_STR_MATCHES(s.ToString(), "client error:.*certificate verify failed");
+
+  // If the client doesn't care, it should succeed against the self-signed
+  // server.
+  ASSERT_OK(RunHandshake(TlsVerificationMode::VERIFY_NONE,
+                         TlsVerificationMode::VERIFY_NONE));
+
+  // If the client loads the cert as trusted, then it should succeed
+  // with verification enabled.
+  ASSERT_OK(client_tls_.LoadCertificateAuthority(cert_path_));
+  ASSERT_OK(RunHandshake(TlsVerificationMode::VERIFY_REMOTE_CERT_AND_HOST,
+                         TlsVerificationMode::VERIFY_NONE));
+
+  // If the server requires authentication of the client, the handshake should fail.
+  s = RunHandshake(TlsVerificationMode::VERIFY_NONE,
+                   TlsVerificationMode::VERIFY_REMOTE_CERT_AND_HOST);
+  ASSERT_TRUE(s.IsRuntimeError());
+  ASSERT_STR_MATCHES(s.ToString(), "server error:.*peer did not return a certificate");
+}
+
+// TODO(PKI): this test has both the client and server using the same cert,
+// which isn't very realistic. We should have this generate self-signed certs
+// on the fly.
+TEST_F(TestTlsHandshake, Test_ClientSelfSigned_ServerSelfSigned) {
+  ASSERT_OK(client_tls_.LoadCertificate(cert_path_));
+  ASSERT_OK(client_tls_.LoadPrivateKey(key_path_));
+  ASSERT_OK(client_tls_.LoadCertificateAuthority(cert_path_));
+  ASSERT_OK(server_tls_.LoadCertificate(cert_path_));
+  ASSERT_OK(server_tls_.LoadPrivateKey(key_path_));
+  ASSERT_OK(server_tls_.LoadCertificateAuthority(cert_path_));
+
+  // This scenario should succeed in all cases.
+  ASSERT_OK(RunHandshake(TlsVerificationMode::VERIFY_REMOTE_CERT_AND_HOST,
+                         TlsVerificationMode::VERIFY_REMOTE_CERT_AND_HOST));
+
+  ASSERT_OK(RunHandshake(TlsVerificationMode::VERIFY_REMOTE_CERT_AND_HOST,
+                         TlsVerificationMode::VERIFY_NONE));
+
+  ASSERT_OK(RunHandshake(TlsVerificationMode::VERIFY_NONE,
+                         TlsVerificationMode::VERIFY_REMOTE_CERT_AND_HOST));
+
+  ASSERT_OK(RunHandshake(TlsVerificationMode::VERIFY_NONE,
+                         TlsVerificationMode::VERIFY_NONE));
+}
+
+// TODO(PKI): add test coverage for mismatched common-name in the cert.
+
 } // namespace security
 } // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/63ce3ec6/src/kudu/security/tls_handshake.cc
----------------------------------------------------------------------
diff --git a/src/kudu/security/tls_handshake.cc b/src/kudu/security/tls_handshake.cc
index 926861f..49f0252 100644
--- a/src/kudu/security/tls_handshake.cc
+++ b/src/kudu/security/tls_handshake.cc
@@ -28,6 +28,7 @@
 #include "kudu/security/tls_socket.h"
 #include "kudu/util/net/sockaddr.h"
 #include "kudu/util/status.h"
+#include "kudu/util/trace.h"
 
 #if OPENSSL_VERSION_NUMBER < 0x10002000L
 #include "kudu/security/x509_check_host.h"
@@ -43,7 +44,47 @@ template<> struct SslTypeTraits<X509> {
   static constexpr auto free = &X509_free;
 };
 
+
+void TlsHandshake::SetSSLVerify() {
+  CHECK(ssl_);
+  CHECK(!has_started_);
+  int ssl_mode = 0;
+  switch (verification_mode_) {
+    case TlsVerificationMode::VERIFY_NONE:
+      ssl_mode = SSL_VERIFY_NONE;
+      break;
+    case TlsVerificationMode::VERIFY_REMOTE_CERT_AND_HOST:
+      // Server mode: the server sends a client certificate request to the client. The
+      // certificate returned (if any) is checked. If the verification process fails, the TLS/SSL
+      // handshake is immediately terminated with an alert message containing the reason for the
+      // verification failure. The behaviour can be controlled by the additional
+      // SSL_VERIFY_FAIL_IF_NO_PEER_CERT and SSL_VERIFY_CLIENT_ONCE flags.
+
+      // Client mode: the server certificate is verified. If the verification process fails, the
+      // TLS/SSL handshake is immediately terminated with an alert message containing the reason
+      // for the verification failure. If no server certificate is sent, because an anonymous
+      // cipher is used, SSL_VERIFY_PEER is ignored.
+      ssl_mode |= SSL_VERIFY_PEER;
+
+      // Server mode: if the client did not return a certificate, the TLS/SSL handshake is
+      // immediately terminated with a "handshake failure" alert. This flag must be used
+      // together with SSL_VERIFY_PEER.
+      ssl_mode |= SSL_VERIFY_FAIL_IF_NO_PEER_CERT;
+      // Server mode: only request a client certificate on the initial TLS/SSL handshake. Do
+      // not ask for a client certificate again in case of a renegotiation. This flag must be
+      // used together with SSL_VERIFY_PEER.
+      ssl_mode |= SSL_VERIFY_CLIENT_ONCE;
+      break;
+  }
+
+  SSL_set_verify(ssl_.get(), ssl_mode, /* callback = */nullptr);
+}
+
 Status TlsHandshake::Continue(const string& recv, string* send) {
+  if (!has_started_) {
+    SetSSLVerify();
+    has_started_ = true;
+  }
   CHECK(ssl_);
   ERR_clear_error();
 
@@ -81,9 +122,13 @@ Status TlsHandshake::Continue(const string& recv, string* send) {
 Status TlsHandshake::Verify(const Socket& socket) const {
   DCHECK(SSL_is_init_finished(ssl_.get()));
   CHECK(ssl_);
-  ERR_clear_error();
 
-  // Verify if the handshake was successful.
+  if (verification_mode_ == TlsVerificationMode::VERIFY_NONE) {
+    return Status::OK();
+  }
+  DCHECK(verification_mode_ == TlsVerificationMode::VERIFY_REMOTE_CERT_AND_HOST);
+
+  ERR_clear_error();
   int rc = SSL_get_verify_result(ssl_.get());
   if (rc != X509_V_OK) {
     return Status::NotAuthorized("SSL_get_verify_result()", X509_verify_cert_error_string(rc));
@@ -91,10 +136,17 @@ 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()));
-  if (!cert && (SSL_get_verify_mode(ssl_.get()) & SSL_VERIFY_FAIL_IF_NO_PEER_CERT)) {
-    return Status::NotAuthorized("Handshake failed: unable to retreive peer certificate");
+  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");
+    }
+    // No cert, but we weren't requiring one.
+    TRACE("Got no cert from peer, but not required");
+    return Status::OK();
   }
 
+  TRACE("Verifying peer cert");
+
   // Get the peer's hostname
   Sockaddr peer_addr;
   if (!socket.GetPeerAddress(&peer_addr).ok()) {

http://git-wip-us.apache.org/repos/asf/kudu/blob/63ce3ec6/src/kudu/security/tls_handshake.h
----------------------------------------------------------------------
diff --git a/src/kudu/security/tls_handshake.h b/src/kudu/security/tls_handshake.h
index e7fcdd2..2eb3781 100644
--- a/src/kudu/security/tls_handshake.h
+++ b/src/kudu/security/tls_handshake.h
@@ -20,7 +20,8 @@
 #include <memory>
 #include <string>
 
-#include "kudu/security/crypto.h"
+#include <glog/logging.h>
+
 #include "kudu/security/openssl_util.h"
 #include "kudu/util/net/socket.h"
 #include "kudu/util/status.h"
@@ -38,6 +39,26 @@ enum class TlsHandshakeType {
   SERVER,
 };
 
+// Mode for performing verification of the remote peer's identity during a handshake.
+enum class TlsVerificationMode {
+  // SERVER:
+  //    No certificate will be requested from the client, and no verification
+  //    will be done.
+  // CLIENT:
+  //    The server's certificate will be obtained but no verification will be done.
+  //    (the server still requires a certificate, even if it is self-signed).
+  VERIFY_NONE,
+
+  // BOTH:
+  // The remote peer is required to have a signed certificate. The certificate will
+  // be verified in two ways:
+  //  1) The certificate must be signed by a trusted CA (or chain of CAs).
+  //  2) Second, the hostname of the remote peer (as determined by reverse DNS of the
+  //    socket address) must match the common name or one of the Subject Alternative
+  //    Names stored in the certificate.
+  VERIFY_REMOTE_CERT_AND_HOST
+};
+
 // TlsHandshake manages an ongoing TLS handshake between a client and server.
 //
 // TlsHandshake instances are default constructed, but must be initialized
@@ -48,6 +69,15 @@ class TlsHandshake {
    TlsHandshake() = default;
    ~TlsHandshake() = default;
 
+  // Set the verification mode for this handshake. The default verification mode
+  // is VERIFY_REMOTE_CERT_AND_HOST.
+  //
+  // This must be not be called after the first call to Continue()
+  void set_verification_mode(TlsVerificationMode mode) {
+    DCHECK(!has_started_);
+    verification_mode_ = mode;
+  }
+
   // Continue or start a new handshake.
   //
   // 'recv' should contain the input buffer from the remote end, or an empty
@@ -77,6 +107,12 @@ class TlsHandshake {
  private:
   friend class TlsContext;
 
+  bool has_started_ = false;
+  TlsVerificationMode verification_mode_ = TlsVerificationMode::VERIFY_REMOTE_CERT_AND_HOST;
+
+  // Set the verification mode on the underlying SSL object.
+  void SetSSLVerify();
+
   // Set the SSL to use during the handshake. Called once by
   // TlsContext::InitiateHandshake before starting the handshake processes.
   void adopt_ssl(c_unique_ptr<SSL> ssl) {