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/01/13 03:01:20 UTC

[1/2] kudu git commit: Move SSLFactory and SSLSocket to the security module

Repository: kudu
Updated Branches:
  refs/heads/master 77e38401c -> f907db137


Move SSLFactory and SSLSocket to the security module

Soon these classes will rely on the security module, so to avoid cyclic
dependencies they must be moved.

Change-Id: I100966cfd51435e2954459fce79baa7cf6da4dcb
Reviewed-on: http://gerrit.cloudera.org:8080/5703
Reviewed-by: Alexey Serbin <as...@cloudera.com>
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/71638f5d
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/71638f5d
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/71638f5d

Branch: refs/heads/master
Commit: 71638f5d604255461747f3dfa917db638492d49c
Parents: 77e3840
Author: Dan Burkert <da...@apache.org>
Authored: Thu Jan 12 15:31:45 2017 -0800
Committer: Dan Burkert <da...@apache.org>
Committed: Fri Jan 13 03:00:37 2017 +0000

----------------------------------------------------------------------
 src/kudu/rpc/CMakeLists.txt          |   9 +-
 src/kudu/rpc/connection.cc           |   6 +-
 src/kudu/rpc/messenger.cc            |   2 +-
 src/kudu/rpc/reactor.cc              |   8 +-
 src/kudu/security/CMakeLists.txt     |  29 +-
 src/kudu/security/ssl_factory.cc     | 154 +++++++++++
 src/kudu/security/ssl_factory.h      |  67 +++++
 src/kudu/security/ssl_socket.cc      | 175 ++++++++++++
 src/kudu/security/ssl_socket.h       |  58 ++++
 src/kudu/security/x509_check_host.cc | 441 ++++++++++++++++++++++++++++++
 src/kudu/security/x509_check_host.h  |  48 ++++
 src/kudu/util/CMakeLists.txt         |  11 -
 src/kudu/util/net/net_util-test.cc   |   1 -
 src/kudu/util/net/ssl_factory.cc     | 153 -----------
 src/kudu/util/net/ssl_factory.h      |  66 -----
 src/kudu/util/net/ssl_socket.cc      | 174 ------------
 src/kudu/util/net/ssl_socket.h       |  60 ----
 src/kudu/util/x509_check_host.cc     | 441 ------------------------------
 src/kudu/util/x509_check_host.h      |  48 ----
 19 files changed, 977 insertions(+), 974 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/71638f5d/src/kudu/rpc/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/CMakeLists.txt b/src/kudu/rpc/CMakeLists.txt
index f35965c..6ba028b 100644
--- a/src/kudu/rpc/CMakeLists.txt
+++ b/src/kudu/rpc/CMakeLists.txt
@@ -70,12 +70,13 @@ set(KRPC_SRCS
 )
 
 set(KRPC_LIBS
-  rpc_header_proto
-  rpc_introspection_proto
-  kudu_util
+  cyrus_sasl
   gutil
+  kudu_util
   libev
-  cyrus_sasl)
+  rpc_header_proto
+  rpc_introspection_proto
+  security)
 
 ADD_EXPORTABLE_LIBRARY(krpc
   SRCS ${KRPC_SRCS}

http://git-wip-us.apache.org/repos/asf/kudu/blob/71638f5d/src/kudu/rpc/connection.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/connection.cc b/src/kudu/rpc/connection.cc
index 8ce98a1..8506b99 100644
--- a/src/kudu/rpc/connection.cc
+++ b/src/kudu/rpc/connection.cc
@@ -30,21 +30,21 @@
 #include "kudu/gutil/map-util.h"
 #include "kudu/gutil/strings/human_readable.h"
 #include "kudu/gutil/strings/substitute.h"
-#include "kudu/rpc/rpc_introspection.pb.h"
 #include "kudu/rpc/constants.h"
 #include "kudu/rpc/messenger.h"
 #include "kudu/rpc/reactor.h"
 #include "kudu/rpc/rpc_controller.h"
 #include "kudu/rpc/rpc_header.pb.h"
+#include "kudu/rpc/rpc_introspection.pb.h"
 #include "kudu/rpc/sasl_client.h"
 #include "kudu/rpc/sasl_server.h"
 #include "kudu/rpc/transfer.h"
+#include "kudu/security/ssl_factory.h"
+#include "kudu/security/ssl_socket.h"
 #include "kudu/util/debug-util.h"
 #include "kudu/util/flag_tags.h"
 #include "kudu/util/logging.h"
 #include "kudu/util/net/sockaddr.h"
-#include "kudu/util/net/ssl_factory.h"
-#include "kudu/util/net/ssl_socket.h"
 #include "kudu/util/status.h"
 #include "kudu/util/trace.h"
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/71638f5d/src/kudu/rpc/messenger.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/messenger.cc b/src/kudu/rpc/messenger.cc
index c5acc20..103f72c 100644
--- a/src/kudu/rpc/messenger.cc
+++ b/src/kudu/rpc/messenger.cc
@@ -42,12 +42,12 @@
 #include "kudu/rpc/rpcz_store.h"
 #include "kudu/rpc/sasl_common.h"
 #include "kudu/rpc/transfer.h"
+#include "kudu/security/ssl_factory.h"
 #include "kudu/util/errno.h"
 #include "kudu/util/flag_tags.h"
 #include "kudu/util/metrics.h"
 #include "kudu/util/monotime.h"
 #include "kudu/util/net/socket.h"
-#include "kudu/util/net/ssl_factory.h"
 #include "kudu/util/status.h"
 #include "kudu/util/threadpool.h"
 #include "kudu/util/trace.h"

http://git-wip-us.apache.org/repos/asf/kudu/blob/71638f5d/src/kudu/rpc/reactor.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/reactor.cc b/src/kudu/rpc/reactor.cc
index 115afb4..0b2b19b 100644
--- a/src/kudu/rpc/reactor.cc
+++ b/src/kudu/rpc/reactor.cc
@@ -39,19 +39,19 @@
 #include "kudu/rpc/sasl_client.h"
 #include "kudu/rpc/sasl_server.h"
 #include "kudu/rpc/transfer.h"
+#include "kudu/security/ssl_factory.h"
+#include "kudu/security/ssl_socket.h"
 #include "kudu/util/countdown_latch.h"
 #include "kudu/util/debug/sanitizer_scopes.h"
 #include "kudu/util/errno.h"
 #include "kudu/util/flag_tags.h"
 #include "kudu/util/monotime.h"
+#include "kudu/util/net/socket.h"
 #include "kudu/util/status.h"
 #include "kudu/util/thread.h"
-#include "kudu/util/threadpool.h"
 #include "kudu/util/thread_restrictions.h"
+#include "kudu/util/threadpool.h"
 #include "kudu/util/trace.h"
-#include "kudu/util/net/socket.h"
-#include "kudu/util/net/ssl_factory.h"
-#include "kudu/util/net/ssl_socket.h"
 
 // When compiling on Mac OS X, use 'kqueue' instead of the default, 'select', for the event loop.
 // Otherwise we run into problems because 'select' can't handle connections when more than 1024

http://git-wip-us.apache.org/repos/asf/kudu/blob/71638f5d/src/kudu/security/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/kudu/security/CMakeLists.txt b/src/kudu/security/CMakeLists.txt
index a4204bc..2577771 100644
--- a/src/kudu/security/CMakeLists.txt
+++ b/src/kudu/security/CMakeLists.txt
@@ -24,16 +24,30 @@ if(NOT APPLE)
   target_link_libraries(krb5_realm_override dl)
 endif()
 
-add_library(security
-  init.cc)
-target_link_libraries(security
+# Fall back to using the ported functionality if we're using an older version of OpenSSL.
+if (${OPENSSL_VERSION} VERSION_LESS "1.0.2")
+  set(PORTED_X509_CHECK_HOST_CC "x509_check_host.cc")
+endif()
+
+set(SECURITY_SRCS
+  init.cc
+  ${PORTED_X509_CHECK_HOST_CC}
+  ssl_factory.cc
+  ssl_socket.cc)
+
+set(SECURITY_LIBS
   gutil
+  krb5
   kudu_util
-  krb5)
+  openssl_crypto
+  openssl_ssl)
+
+ADD_EXPORTABLE_LIBRARY(security
+  SRCS ${SECURITY_SRCS}
+  DEPS ${SECURITY_LIBS})
 
 set(SECURITY_TEST_SRCS
-  test/mini_kdc.cc
-)
+  test/mini_kdc.cc)
 
 add_library(security-test ${SECURITY_TEST_SRCS})
 target_link_libraries(security-test
@@ -41,11 +55,10 @@ target_link_libraries(security-test
   kudu_test_util
   kudu_util)
 
-
 # Tests
 set(KUDU_TEST_LINK_LIBS
   security
   security-test
   ${KUDU_MIN_TEST_LIBS})
 
-ADD_KUDU_TEST(test/mini_kdc-test)
\ No newline at end of file
+ADD_KUDU_TEST(test/mini_kdc-test)

http://git-wip-us.apache.org/repos/asf/kudu/blob/71638f5d/src/kudu/security/ssl_factory.cc
----------------------------------------------------------------------
diff --git a/src/kudu/security/ssl_factory.cc b/src/kudu/security/ssl_factory.cc
new file mode 100644
index 0000000..97de04b
--- /dev/null
+++ b/src/kudu/security/ssl_factory.cc
@@ -0,0 +1,154 @@
+// 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/ssl_factory.h"
+
+#include <mutex>
+#include <vector>
+
+#include <openssl/err.h>
+#include <openssl/rand.h>
+#include <openssl/ssl.h>
+#include <openssl/x509v3.h>
+
+#include "kudu/security/ssl_socket.h"
+#include "kudu/util/debug/leakcheck_disabler.h"
+#include "kudu/util/mutex.h"
+#include "kudu/util/thread.h"
+
+namespace kudu {
+
+// These non-POD elements will be alive for the lifetime of the process, so don't allocate in
+// static storage.
+static Mutex* g_ssl_mutexes;
+
+// Lock/Unlock the nth lock. Only to be used by OpenSSL.
+static void CryptoLockingCallback(int mode, int n, const char* /*unused*/, int /*unused*/) {
+  if (mode & CRYPTO_LOCK) {
+    g_ssl_mutexes[n].Acquire();
+  } else {
+    g_ssl_mutexes[n].Release();
+  }
+}
+
+// Return the current pthread's tid. Only to be used by OpenSSL.
+static void CryptoThreadIDCallback(CRYPTO_THREADID* id) {
+  return CRYPTO_THREADID_set_numeric(id, Thread::UniqueThreadId());
+}
+
+void DoSSLInit() {
+  SSL_library_init();
+  SSL_load_error_strings();
+  OpenSSL_add_all_algorithms();
+  RAND_poll();
+
+  debug::ScopedLeakCheckDisabler d;
+  g_ssl_mutexes = new Mutex[CRYPTO_num_locks()];
+
+  // Callbacks used by OpenSSL required in a multi-threaded setting.
+  CRYPTO_set_locking_callback(CryptoLockingCallback);
+  CRYPTO_THREADID_set_callback(CryptoThreadIDCallback);
+}
+
+SSLFactory::SSLFactory() : ctx_(nullptr, SSL_CTX_free) {
+  static std::once_flag ssl_once;
+  std::call_once(ssl_once, DoSSLInit);
+}
+
+SSLFactory::~SSLFactory() {
+}
+
+Status SSLFactory::Init() {
+  CHECK(!ctx_.get());
+  // NOTE: 'SSLv23 method' sounds like it would enable only SSLv2 and SSLv3, but in fact
+  // this is a sort of wildcard which enables all methods (including TLSv1 and later).
+  // We explicitly disable SSLv2 and SSLv3 below so that only TLS methods remain.
+  // See the discussion on https://trac.torproject.org/projects/tor/ticket/11598 for more
+  // info.
+  ctx_.reset(SSL_CTX_new(SSLv23_method()));
+  if (!ctx_) {
+    return Status::RuntimeError("Could not create SSL context");
+  }
+  SSL_CTX_set_mode(ctx_.get(), SSL_MODE_AUTO_RETRY);
+
+  // Disable SSLv2 and SSLv3 which are vulnerable to various issues such as POODLE.
+  // We support versions back to TLSv1.0 since OpenSSL on RHEL 6.4 and earlier does not
+  // not support TLSv1.1 or later.
+  //
+  // Disable SSL/TLS compression to free up CPU resources and be less prone
+  // to attacks exploiting the compression feature:
+  //   https://tools.ietf.org/html/rfc7525#section-3.3
+  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);
+  return Status::OK();
+}
+
+std::string SSLFactory::GetLastError(int errno_copy) {
+  int error_code = ERR_get_error();
+  if (error_code == 0) return kudu::ErrnoToString(errno_copy);
+  const char* error_reason = ERR_reason_error_string(error_code);
+  if (error_reason != NULL) return error_reason;
+  return strings::Substitute("SSL error $0", error_code);
+}
+
+Status SSLFactory::LoadCertificate(const std::string& certificate_path) {
+  ERR_clear_error();
+  errno = 0;
+  if (SSL_CTX_use_certificate_file(ctx_.get(), certificate_path.c_str(), SSL_FILETYPE_PEM) != 1) {
+    return Status::NotFound(
+        "Failed to load certificate file '" + certificate_path + "': " + GetLastError(errno));
+  }
+  return Status::OK();
+}
+
+Status SSLFactory::LoadPrivateKey(const std::string& key_path) {
+  ERR_clear_error();
+  errno = 0;
+  if (SSL_CTX_use_PrivateKey_file(ctx_.get(), key_path.c_str(), SSL_FILETYPE_PEM) != 1) {
+    return Status::NotFound(
+        "Failed to load private key file '" + key_path + "': " + GetLastError(errno));
+  }
+  return Status::OK();
+}
+
+Status SSLFactory::LoadCertificateAuthority(const std::string& certificate_path) {
+  ERR_clear_error();
+  errno = 0;
+  if (SSL_CTX_load_verify_locations(ctx_.get(), certificate_path.c_str(), nullptr) != 1) {
+    return Status::NotFound(
+        "Failed to load certificate authority file '" + certificate_path + "': " +
+            GetLastError(errno));
+  }
+  return Status::OK();
+}
+
+std::unique_ptr<SSLSocket> SSLFactory::CreateSocket(int socket_fd, bool is_server) {
+  CHECK(ctx_);
+  // Create SSL object and transfer ownership to the SSLSocket object created.
+  SSL* ssl = SSL_new(ctx_.get());
+  if (ssl == nullptr) {
+    return nullptr;
+  }
+  std::unique_ptr<SSLSocket> socket(new SSLSocket(socket_fd, ssl, is_server));
+  return socket;
+  //return new SSLSocket(socket_fd, ssl, is_server);
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/71638f5d/src/kudu/security/ssl_factory.h
----------------------------------------------------------------------
diff --git a/src/kudu/security/ssl_factory.h b/src/kudu/security/ssl_factory.h
new file mode 100644
index 0000000..9d2dd31
--- /dev/null
+++ b/src/kudu/security/ssl_factory.h
@@ -0,0 +1,67 @@
+// 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 <memory>
+
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/util/errno.h"
+#include "kudu/util/status.h"
+
+struct ssl_ctx_st;
+typedef ssl_ctx_st SSL_CTX;
+
+namespace kudu {
+
+class Sockaddr;
+class SSLSocket;
+
+class SSLFactory {
+ public:
+  SSLFactory();
+
+  ~SSLFactory();
+
+  // Set up the SSL_CTX and choose encryption preferences.
+  Status Init();
+
+  // Load the server certificate.
+  Status LoadCertificate(const std::string& certificate_path);
+
+  // Load the private key for the server certificate.
+  Status LoadPrivateKey(const std::string& key_path);
+
+  // Load the certificate authority.
+  Status LoadCertificateAuthority(const std::string& certificate_path);
+
+  // Create an SSLSocket wrapped around the file descriptor 'socket_fd'. 'is_server' denotes if it's
+  // a server socket. The 'socket_fd' is closed when this object is destroyed.
+  std::unique_ptr<SSLSocket> CreateSocket(int socket_fd, bool is_server);
+
+ private:
+  friend class SSLSocket;
+  std::unique_ptr<SSL_CTX, std::function<void(SSL_CTX*)>> ctx_;
+
+  // Gets the last error from the thread local SSL error queue. If no error exists, it returns
+  // the error corresponding to 'errno_copy'.
+  static std::string GetLastError(int errno_copy);
+};
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/71638f5d/src/kudu/security/ssl_socket.cc
----------------------------------------------------------------------
diff --git a/src/kudu/security/ssl_socket.cc b/src/kudu/security/ssl_socket.cc
new file mode 100644
index 0000000..126d2f1
--- /dev/null
+++ b/src/kudu/security/ssl_socket.cc
@@ -0,0 +1,175 @@
+// 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/ssl_socket.h"
+
+#include <errno.h>
+
+#include <vector>
+
+#include <openssl/err.h>
+#include <openssl/ssl.h>
+#include <openssl/x509.h>
+#include <openssl/x509v3.h>
+
+#include "kudu/gutil/strings/split.h"
+#include "kudu/gutil/strings/util.h"
+#include "kudu/security/ssl_factory.h"
+#include "kudu/util/errno.h"
+#include "kudu/util/net/sockaddr.h"
+
+#if OPENSSL_VERSION_NUMBER < 0x10002000L
+#include "kudu/security/x509_check_host.h"
+#endif // OPENSSL_VERSION_NUMBER
+
+namespace kudu {
+
+SSLSocket::SSLSocket(int fd, SSL* ssl, bool is_server) :
+    Socket(fd), ssl_(ssl), is_server_(is_server) {
+  SSL_set_fd(ssl_, fd);
+}
+
+SSLSocket::~SSLSocket() {
+}
+
+Status SSLSocket::DoHandshake() {
+  CHECK(ssl_);
+  ERR_clear_error();
+  errno = 0;
+  int ret;
+  if (is_server_) {
+    ret = SSL_accept(ssl_);
+  } else {
+    ret = SSL_connect(ssl_);
+  }
+  if (ret <= 0) return Status::NetworkError(SSLFactory::GetLastError(errno));
+
+  // Verify if the handshake was successful.
+  int rc = SSL_get_verify_result(ssl_);
+  if (rc != X509_V_OK) {
+    return Status::NetworkError("SSL_get_verify_result()", X509_verify_cert_error_string(rc));
+  }
+
+  // Get the peer certificate.
+  std::unique_ptr<X509, void(*)(X509*)> cert(
+      SSL_get_peer_certificate(ssl_), [](X509* x) { X509_free(x); });
+  cert.reset(SSL_get_peer_certificate(ssl_));
+  if (cert == nullptr) {
+    if (SSL_get_verify_mode(ssl_) & SSL_VERIFY_FAIL_IF_NO_PEER_CERT) {
+      return Status::NetworkError("Handshake failed: Could not retreive peer certificate");
+    }
+  }
+
+  // Get the peer's hostname
+  Sockaddr peer_addr;
+  if (!GetPeerAddress(&peer_addr).ok()) {
+    return Status::NetworkError("Handshake failed: Could not retrieve peer address");
+  }
+  std::string peer_hostname;
+  RETURN_NOT_OK(peer_addr.LookupHostname(&peer_hostname));
+
+  // Check if the hostname matches with either the Common Name or any of the Subject Alternative
+  // Names of the certificate.
+  int match;
+  if ((match = X509_check_host(
+      cert.get(), peer_hostname.c_str(), peer_hostname.length(), 0, nullptr)) == 0) {
+    return Status::NetworkError("Handshake failed: Could not verify host with certificate");
+  }
+  if (match < 0) {
+    return Status::NetworkError("Handshake failed:", SSLFactory::GetLastError(errno));
+  }
+  CHECK(match == 1);
+  return Status::OK();
+}
+
+Status SSLSocket::Write(const uint8_t *buf, int32_t amt, int32_t *nwritten) {
+  CHECK(ssl_);
+  ERR_clear_error();
+  errno = 0;
+  int32_t bytes_written = SSL_write(ssl_, buf, amt);
+  if (bytes_written <= 0) {
+    if (SSL_get_error(ssl_, bytes_written) == SSL_ERROR_WANT_WRITE) {
+      // Socket not ready to write yet.
+      *nwritten = 0;
+      return Status::OK();
+    }
+    return Status::NetworkError("SSL_write", SSLFactory::GetLastError(errno));
+  }
+  *nwritten = bytes_written;
+  return Status::OK();
+}
+
+Status SSLSocket::Writev(const struct ::iovec *iov, int iov_len,
+                      int32_t *nwritten) {
+  CHECK(ssl_);
+  ERR_clear_error();
+  int32_t total_written = 0;
+  // Allows packets to be aggresively be accumulated before sending.
+  RETURN_NOT_OK(SetTcpCork(1));
+  Status write_status = Status::OK();
+  for (int i = 0; i < iov_len; ++i) {
+    int32_t frame_size = iov[i].iov_len;
+    // Don't return before unsetting TCP_CORK.
+    write_status = Write(static_cast<uint8_t*>(iov[i].iov_base), frame_size, nwritten);
+    total_written += *nwritten;
+    if (*nwritten < frame_size) break;
+  }
+  RETURN_NOT_OK(SetTcpCork(0));
+  *nwritten = total_written;
+  return write_status;
+}
+
+Status SSLSocket::Recv(uint8_t *buf, int32_t amt, int32_t *nread) {
+  CHECK(ssl_);
+  ERR_clear_error();
+  errno = 0;
+  int32_t bytes_read = SSL_read(ssl_, buf, amt);
+  if (bytes_read <= 0) {
+    if (bytes_read == 0 && SSL_get_shutdown(ssl_) == SSL_RECEIVED_SHUTDOWN) {
+      return Status::NetworkError("SSLSocket::Recv() for EOF from remote", Slice(), ESHUTDOWN);
+    }
+    if (SSL_get_error(ssl_, bytes_read) == SSL_ERROR_WANT_READ) {
+      // Nothing available to read yet.
+      *nread = 0;
+      return Status::OK();
+    }
+    return Status::NetworkError("SSL_read", SSLFactory::GetLastError(errno));
+  }
+  *nread = bytes_read;
+  return Status::OK();
+}
+
+Status SSLSocket::Close() {
+  CHECK(ssl_);
+  ERR_clear_error();
+  errno = 0;
+  int32_t ret = SSL_shutdown(ssl_);
+  Status shutdown_status;
+  if (ret < 0 && errno != EAGAIN) {
+    // We still need to close the underlying socket, so don't return just yet.
+    shutdown_status = Status::NetworkError("SSL_Shutdown", SSLFactory::GetLastError(errno));
+  }
+  SSL_free(ssl_);
+  ssl_ = nullptr;
+  ERR_remove_state(0);
+
+  Status close_status = Socket::Close();
+  if (!close_status.ok()) return close_status.CloneAndPrepend(shutdown_status.message());
+  return shutdown_status;
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/71638f5d/src/kudu/security/ssl_socket.h
----------------------------------------------------------------------
diff --git a/src/kudu/security/ssl_socket.h b/src/kudu/security/ssl_socket.h
new file mode 100644
index 0000000..91fe39f
--- /dev/null
+++ b/src/kudu/security/ssl_socket.h
@@ -0,0 +1,58 @@
+// 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 <sys/uio.h>
+
+#include <string>
+
+#include "kudu/gutil/macros.h"
+#include "kudu/util/net/socket.h"
+#include "kudu/util/status.h"
+
+struct ssl_st;
+typedef ssl_st SSL;
+
+namespace kudu {
+
+class Sockaddr;
+
+class SSLSocket : public Socket {
+ public:
+  SSLSocket(int fd, SSL* ssl, bool is_server);
+
+  ~SSLSocket();
+
+  // Do the SSL handshake as a client or a server and verify that the credentials were correctly
+  // verified.
+  Status DoHandshake();
+
+  Status Write(const uint8_t *buf, int32_t amt, int32_t *nwritten) override;
+
+  Status Writev(const struct ::iovec *iov, int iov_len, int32_t *nwritten) override;
+
+  Status Recv(uint8_t *buf, int32_t amt, int32_t *nread) override;
+
+  // Shutdown the connection and free the SSL state for this connection.
+  Status Close() override;
+ private:
+  SSL* ssl_; // Owned.
+  bool is_server_;
+};
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/71638f5d/src/kudu/security/x509_check_host.cc
----------------------------------------------------------------------
diff --git a/src/kudu/security/x509_check_host.cc b/src/kudu/security/x509_check_host.cc
new file mode 100644
index 0000000..ce59138
--- /dev/null
+++ b/src/kudu/security/x509_check_host.cc
@@ -0,0 +1,441 @@
+/*
+ * Copyright 1999-2016 The OpenSSL Project Authors. All Rights Reserved.
+ *
+ * Licensed under the OpenSSL license (the "License").  You may not use
+ * this file except in compliance with the License.  You can obtain a copy
+ * in the file LICENSE in the source distribution or at
+ * https://www.openssl.org/source/license.html
+ */
+
+// The following is ported from the OpenSSL-1.1.0b library. The implementations
+// of the functions are for the most part the same except where mentioned in special
+// comments. Explicit casts were also added to bypass compilation errors.
+
+#include <stdio.h>
+#include <ctype.h>
+#include <string.h>
+#include <strings.h>
+
+#include <openssl/bn.h>
+#include <openssl/conf.h>
+#include <openssl/err.h>
+#include <openssl/x509v3.h>
+
+#include "kudu/security/x509_check_host.h"
+
+// Ported from include/openssl/crypto.h from OpenSSL-1.1.0b
+// Modifed to use __FILE__ and __LINE__ instead of OPENSSL_FILE and OPENSSL_LINE.
+# define OPENSSL_strndup(str, n) \
+        CRYPTO_strndup(str, n, __FILE__, __LINE__)
+
+// Ported from crypto/o_str.c from OpenSSL-1.1.0b.
+// Modified to use strnlen() instead of OPENSSL_strnlen()
+char *CRYPTO_strndup(const char *str, size_t s, const char* file, int line)
+{
+    size_t maxlen;
+    char *ret;
+
+    if (str == NULL)
+        return NULL;
+
+    maxlen = strnlen(str, s);
+
+    ret = (char*)CRYPTO_malloc(maxlen + 1, file, line);
+    if (ret) {
+        memcpy(ret, str, maxlen);
+        ret[maxlen] = '\0';
+    }
+    return ret;
+}
+
+// The remaining code is ported form crypto/x509v3/v3_utl.c
+
+typedef int (*equal_fn) (const unsigned char *pattern, size_t pattern_len,
+                         const unsigned char *subject, size_t subject_len,
+                         unsigned int flags);
+
+/* Skip pattern prefix to match "wildcard" subject */
+static void skip_prefix(const unsigned char **p, size_t *plen,
+                        size_t subject_len,
+                        unsigned int flags)
+{
+    const unsigned char *pattern = *p;
+    size_t pattern_len = *plen;
+
+    /*
+     * If subject starts with a leading '.' followed by more octets, and
+     * pattern is longer, compare just an equal-length suffix with the
+     * full subject (starting at the '.'), provided the prefix contains
+     * no NULs.
+     */
+    if ((flags & _X509_CHECK_FLAG_DOT_SUBDOMAINS) == 0)
+        return;
+
+    while (pattern_len > subject_len && *pattern) {
+        if ((flags & X509_CHECK_FLAG_SINGLE_LABEL_SUBDOMAINS) &&
+            *pattern == '.')
+            break;
+        ++pattern;
+        --pattern_len;
+    }
+
+    /* Skip if entire prefix acceptable */
+    if (pattern_len == subject_len) {
+        *p = pattern;
+        *plen = pattern_len;
+    }
+}
+
+/* Compare while ASCII ignoring case. */
+static int equal_nocase(const unsigned char *pattern, size_t pattern_len,
+                        const unsigned char *subject, size_t subject_len,
+                        unsigned int flags)
+{
+    skip_prefix(&pattern, &pattern_len, subject_len, flags);
+    if (pattern_len != subject_len)
+        return 0;
+    while (pattern_len) {
+        unsigned char l = *pattern;
+        unsigned char r = *subject;
+        /* The pattern must not contain NUL characters. */
+        if (l == 0)
+            return 0;
+        if (l != r) {
+            if ('A' <= l && l <= 'Z')
+                l = (l - 'A') + 'a';
+            if ('A' <= r && r <= 'Z')
+                r = (r - 'A') + 'a';
+            if (l != r)
+                return 0;
+        }
+        ++pattern;
+        ++subject;
+        --pattern_len;
+    }
+    return 1;
+}
+
+/* Compare using memcmp. */
+static int equal_case(const unsigned char *pattern, size_t pattern_len,
+                      const unsigned char *subject, size_t subject_len,
+                      unsigned int flags)
+{
+    skip_prefix(&pattern, &pattern_len, subject_len, flags);
+    if (pattern_len != subject_len)
+        return 0;
+    return !memcmp(pattern, subject, pattern_len);
+}
+
+/*
+ * RFC 5280, section 7.5, requires that only the domain is compared in a
+ * case-insensitive manner.
+ */
+static int equal_email(const unsigned char *a, size_t a_len,
+                       const unsigned char *b, size_t b_len,
+                       unsigned int unused_flags)
+{
+    size_t i = a_len;
+    if (a_len != b_len)
+        return 0;
+    /*
+     * We search backwards for the '@' character, so that we do not have to
+     * deal with quoted local-parts.  The domain part is compared in a
+     * case-insensitive manner.
+     */
+    while (i > 0) {
+        --i;
+        if (a[i] == '@' || b[i] == '@') {
+            if (!equal_nocase(a + i, a_len - i, b + i, a_len - i, 0))
+                return 0;
+            break;
+        }
+    }
+    if (i == 0)
+        i = a_len;
+    return equal_case(a, i, b, i, 0);
+}
+
+/*
+ * Compare the prefix and suffix with the subject, and check that the
+ * characters in-between are valid.
+ */
+static int wildcard_match(const unsigned char *prefix, size_t prefix_len,
+                          const unsigned char *suffix, size_t suffix_len,
+                          const unsigned char *subject, size_t subject_len,
+                          unsigned int flags)
+{
+    const unsigned char *wildcard_start;
+    const unsigned char *wildcard_end;
+    const unsigned char *p;
+    int allow_multi = 0;
+    int allow_idna = 0;
+
+    if (subject_len < prefix_len + suffix_len)
+        return 0;
+    if (!equal_nocase(prefix, prefix_len, subject, prefix_len, flags))
+        return 0;
+    wildcard_start = subject + prefix_len;
+    wildcard_end = subject + (subject_len - suffix_len);
+    if (!equal_nocase(wildcard_end, suffix_len, suffix, suffix_len, flags))
+        return 0;
+    /*
+     * If the wildcard makes up the entire first label, it must match at
+     * least one character.
+     */
+    if (prefix_len == 0 && *suffix == '.') {
+        if (wildcard_start == wildcard_end)
+            return 0;
+        allow_idna = 1;
+        if (flags & X509_CHECK_FLAG_MULTI_LABEL_WILDCARDS)
+            allow_multi = 1;
+    }
+    /* IDNA labels cannot match partial wildcards */
+    if (!allow_idna &&
+        subject_len >= 4 && strncasecmp((char *)subject, "xn--", 4) == 0)
+        return 0;
+    /* The wildcard may match a literal '*' */
+    if (wildcard_end == wildcard_start + 1 && *wildcard_start == '*')
+        return 1;
+    /*
+     * Check that the part matched by the wildcard contains only
+     * permitted characters and only matches a single label unless
+     * allow_multi is set.
+     */
+    for (p = wildcard_start; p != wildcard_end; ++p)
+        if (!(('0' <= *p && *p <= '9') ||
+              ('A' <= *p && *p <= 'Z') ||
+              ('a' <= *p && *p <= 'z') ||
+              *p == '-' || (allow_multi && *p == '.')))
+            return 0;
+    return 1;
+}
+
+#define LABEL_START     (1 << 0)
+#define LABEL_END       (1 << 1)
+#define LABEL_HYPHEN    (1 << 2)
+#define LABEL_IDNA      (1 << 3)
+
+static const unsigned char *valid_star(const unsigned char *p, size_t len,
+                                       unsigned int flags)
+{
+    const unsigned char *star = 0;
+    size_t i;
+    int state = LABEL_START;
+    int dots = 0;
+    for (i = 0; i < len; ++i) {
+        /*
+         * Locate first and only legal wildcard, either at the start
+         * or end of a non-IDNA first and not final label.
+         */
+        if (p[i] == '*') {
+            int atstart = (state & LABEL_START);
+            int atend = (i == len - 1 || p[i + 1] == '.');
+            /*-
+             * At most one wildcard per pattern.
+             * No wildcards in IDNA labels.
+             * No wildcards after the first label.
+             */
+            if (star != NULL || (state & LABEL_IDNA) != 0 || dots)
+                return NULL;
+            /* Only full-label '*.example.com' wildcards? */
+            if ((flags & X509_CHECK_FLAG_NO_PARTIAL_WILDCARDS)
+                && (!atstart || !atend))
+                return NULL;
+            /* No 'foo*bar' wildcards */
+            if (!atstart && !atend)
+                return NULL;
+            star = &p[i];
+            state &= ~LABEL_START;
+        } else if (('a' <= p[i] && p[i] <= 'z')
+                   || ('A' <= p[i] && p[i] <= 'Z')
+                   || ('0' <= p[i] && p[i] <= '9')) {
+            if ((state & LABEL_START) != 0
+                && len - i >= 4 && strncasecmp((char *)&p[i], "xn--", 4) == 0)
+                state |= LABEL_IDNA;
+            state &= ~(LABEL_HYPHEN | LABEL_START);
+        } else if (p[i] == '.') {
+            if ((state & (LABEL_HYPHEN | LABEL_START)) != 0)
+                return NULL;
+            state = LABEL_START;
+            ++dots;
+        } else if (p[i] == '-') {
+            /* no domain/subdomain starts with '-' */
+            if ((state & LABEL_START) != 0)
+                return NULL;
+            state |= LABEL_HYPHEN;
+        } else
+            return NULL;
+    }
+
+    /*
+     * The final label must not end in a hyphen or ".", and
+     * there must be at least two dots after the star.
+     */
+    if ((state & (LABEL_START | LABEL_HYPHEN)) != 0 || dots < 2)
+        return NULL;
+    return star;
+}
+
+/* Compare using wildcards. */
+static int equal_wildcard(const unsigned char *pattern, size_t pattern_len,
+                          const unsigned char *subject, size_t subject_len,
+                          unsigned int flags)
+{
+    const unsigned char *star = NULL;
+
+    /*
+     * Subject names starting with '.' can only match a wildcard pattern
+     * via a subject sub-domain pattern suffix match.
+     */
+    if (!(subject_len > 1 && subject[0] == '.'))
+        star = valid_star(pattern, pattern_len, flags);
+    if (star == NULL)
+        return equal_nocase(pattern, pattern_len,
+                            subject, subject_len, flags);
+    return wildcard_match(pattern, star - pattern,
+                          star + 1, (pattern + pattern_len) - star - 1,
+                          subject, subject_len, flags);
+}
+
+/*
+ * Compare an ASN1_STRING to a supplied string. If they match return 1. If
+ * cmp_type > 0 only compare if string matches the type, otherwise convert it
+ * to UTF8.
+ */
+
+static int do_check_string(const ASN1_STRING *a, int cmp_type, equal_fn equal,
+                           unsigned int flags, const char *b, size_t blen,
+                           char **peername)
+{
+    int rv = 0;
+
+    if (!a->data || !a->length)
+        return 0;
+    if (cmp_type > 0) {
+        if (cmp_type != a->type)
+            return 0;
+        if (cmp_type == V_ASN1_IA5STRING)
+            rv = equal(a->data, a->length, (unsigned char *)b, blen, flags);
+        else if (a->length == (int)blen && !memcmp(a->data, b, blen))
+            rv = 1;
+        if (rv > 0 && peername)
+            *peername = OPENSSL_strndup((char *)a->data, a->length);
+    } else {
+        int astrlen;
+        unsigned char *astr;
+        astrlen = ASN1_STRING_to_UTF8(&astr, (ASN1_STRING*)a);
+        if (astrlen < 0) {
+            /*
+             * -1 could be an internal malloc failure or a decoding error from
+             * malformed input; we can't distinguish.
+             */
+            return -1;
+        }
+        rv = equal(astr, astrlen, (unsigned char *)b, blen, flags);
+        if (rv > 0 && peername)
+            *peername = OPENSSL_strndup((char *)astr, astrlen);
+            //*peername = strndup((char *)astr, astrlen);
+        OPENSSL_free(astr);
+    }
+    return rv;
+}
+
+static int do_x509_check(X509 *x, const char *chk, size_t chklen,
+                         unsigned int flags, int check_type, char **peername)
+{
+    GENERAL_NAMES *gens = NULL;
+    X509_NAME *name = NULL;
+    int i;
+    int cnid = NID_undef;
+    int alt_type;
+    int san_present = 0;
+    int rv = 0;
+    equal_fn equal;
+
+    /* See below, this flag is internal-only */
+    flags &= ~_X509_CHECK_FLAG_DOT_SUBDOMAINS;
+    if (check_type == GEN_EMAIL) {
+        cnid = NID_pkcs9_emailAddress;
+        alt_type = V_ASN1_IA5STRING;
+        equal = equal_email;
+    } else if (check_type == GEN_DNS) {
+        cnid = NID_commonName;
+        /* Implicit client-side DNS sub-domain pattern */
+        if (chklen > 1 && chk[0] == '.')
+            flags |= _X509_CHECK_FLAG_DOT_SUBDOMAINS;
+        alt_type = V_ASN1_IA5STRING;
+        if (flags & X509_CHECK_FLAG_NO_WILDCARDS)
+            equal = equal_nocase;
+        else
+            equal = equal_wildcard;
+    } else {
+        alt_type = V_ASN1_OCTET_STRING;
+        equal = equal_case;
+    }
+
+    if (chklen == 0)
+        chklen = strlen(chk);
+
+    gens = (GENERAL_NAMES*)X509_get_ext_d2i(x, NID_subject_alt_name, NULL, NULL);
+    if (gens) {
+        for (i = 0; i < sk_GENERAL_NAME_num(gens); i++) {
+            GENERAL_NAME *gen;
+            ASN1_STRING *cstr;
+            gen = sk_GENERAL_NAME_value(gens, i);
+            if (gen->type != check_type)
+                continue;
+            san_present = 1;
+            if (check_type == GEN_EMAIL)
+                cstr = gen->d.rfc822Name;
+            else if (check_type == GEN_DNS)
+                cstr = gen->d.dNSName;
+            else
+                cstr = gen->d.iPAddress;
+            /* Positive on success, negative on error! */
+            if ((rv = do_check_string(cstr, alt_type, equal, flags,
+                                      chk, chklen, peername)) != 0)
+                break;
+        }
+        GENERAL_NAMES_free(gens);
+        if (rv != 0)
+            return rv;
+        if (san_present && !(flags & X509_CHECK_FLAG_ALWAYS_CHECK_SUBJECT))
+            return 0;
+    }
+
+    /* We're done if CN-ID is not pertinent */
+    if (cnid == NID_undef || (flags & X509_CHECK_FLAG_NEVER_CHECK_SUBJECT))
+        return 0;
+
+    i = -1;
+    name = X509_get_subject_name(x);
+    while ((i = X509_NAME_get_index_by_NID(name, cnid, i)) >= 0) {
+        const X509_NAME_ENTRY *ne = X509_NAME_get_entry(name, i);
+        const ASN1_STRING *str = X509_NAME_ENTRY_get_data((X509_NAME_ENTRY*)ne);
+
+        /* Positive on success, negative on error! */
+        if ((rv = do_check_string(str, -1, equal, flags,
+                                  chk, chklen, peername)) != 0)
+            return rv;
+    }
+    return 0;
+}
+
+int X509_check_host(X509 *x, const char *chk, size_t chklen,
+                    unsigned int flags, char **peername)
+{
+    if (chk == NULL)
+        return -2;
+    /*
+     * Embedded NULs are disallowed, except as the last character of a
+     * string of length 2 or more (tolerate caller including terminating
+     * NUL in string length).
+     */
+    if (chklen == 0)
+        chklen = strlen(chk);
+    else if (memchr(chk, '\0', chklen > 1 ? chklen - 1 : chklen))
+        return -2;
+    if (chklen > 1 && chk[chklen - 1] == '\0')
+        --chklen;
+    return do_x509_check(x, chk, chklen, flags, GEN_DNS, peername);
+}

http://git-wip-us.apache.org/repos/asf/kudu/blob/71638f5d/src/kudu/security/x509_check_host.h
----------------------------------------------------------------------
diff --git a/src/kudu/security/x509_check_host.h b/src/kudu/security/x509_check_host.h
new file mode 100644
index 0000000..e388b11
--- /dev/null
+++ b/src/kudu/security/x509_check_host.h
@@ -0,0 +1,48 @@
+/*
+ * Copyright 1999-2016 The OpenSSL Project Authors. All Rights Reserved.
+ *
+ * Licensed under the OpenSSL license (the "License").  You may not use
+ * this file except in compliance with the License.  You can obtain a copy
+ * in the file LICENSE in the source distribution or at
+ * https://www.openssl.org/source/license.html
+ */
+
+// The following is ported from the OpenSSL-1.1.0b library.
+
+#ifndef X509_CHECK_HOST_H
+#define X509_CHECK_HOST_H
+
+#include <stdlib.h>
+
+#include <openssl/x509.h>
+
+/* Flags for X509_check_* functions */
+
+/*
+ * Always check subject name for host match even if subject alt names present
+ */
+# define X509_CHECK_FLAG_ALWAYS_CHECK_SUBJECT    0x1
+/* Disable wildcard matching for dnsName fields and common name. */
+# define X509_CHECK_FLAG_NO_WILDCARDS    0x2
+/* Wildcards must not match a partial label. */
+# define X509_CHECK_FLAG_NO_PARTIAL_WILDCARDS 0x4
+/* Allow (non-partial) wildcards to match multiple labels. */
+# define X509_CHECK_FLAG_MULTI_LABEL_WILDCARDS 0x8
+/* Constraint verifier subdomain patterns to match a single labels. */
+# define X509_CHECK_FLAG_SINGLE_LABEL_SUBDOMAINS 0x10
+/* Never check the subject CN */
+# define X509_CHECK_FLAG_NEVER_CHECK_SUBJECT    0x20
+/*
+ * Match reference identifiers starting with "." to any sub-domain.
+ * This is a non-public flag, turned on implicitly when the subject
+ * reference identity is a DNS name.
+ */
+# define _X509_CHECK_FLAG_DOT_SUBDOMAINS 0x8000
+
+// Checks if the certificate Subject Alternative Name (SAN) or Subject CommonName (CN)
+// matches the specified host name, which must be encoded in the preferred name syntax
+// described in section 3.5 of RFC 1034.
+int X509_check_host(X509 *x, const char *chk, size_t chklen,
+                    unsigned int flags, char **peername);
+
+#endif // X509_CHECK_HOST_H

http://git-wip-us.apache.org/repos/asf/kudu/blob/71638f5d/src/kudu/util/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/kudu/util/CMakeLists.txt b/src/kudu/util/CMakeLists.txt
index d6fd37d..8138972 100644
--- a/src/kudu/util/CMakeLists.txt
+++ b/src/kudu/util/CMakeLists.txt
@@ -101,12 +101,6 @@ else ()
   set(SEMAPHORE_CC "semaphore.cc")
 endif()
 
-# Fall back to using the ported functionality if we're using an older version of
-# OpenSSL
-if (${OPENSSL_VERSION} VERSION_LESS "1.0.2")
-  set(PORTED_X509_CHECK_HOST_CC "x509_check_host.cc")
-endif()
-
 set(UTIL_SRCS
   async_logger.cc
   atomic.cc
@@ -154,15 +148,12 @@ set(UTIL_SRCS
   net/net_util.cc
   net/sockaddr.cc
   net/socket.cc
-  net/ssl_factory.cc
-  net/ssl_socket.cc
   oid_generator.cc
   once.cc
   os-util.cc
   path_util.cc
   pb_util.cc
   pb_util-internal.cc
-  ${PORTED_X509_CHECK_HOST_CC}
   random_util.cc
   resettable_heartbeater.cc
   rolling_log.cc
@@ -211,8 +202,6 @@ set(UTIL_LIBS
   histogram_proto
   libev
   maintenance_manager_proto
-  openssl_crypto
-  openssl_ssl
   pb_util_proto
   protobuf
   version_info_proto

http://git-wip-us.apache.org/repos/asf/kudu/blob/71638f5d/src/kudu/util/net/net_util-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/net/net_util-test.cc b/src/kudu/util/net/net_util-test.cc
index e7bb6c0..b1c33ef 100644
--- a/src/kudu/util/net/net_util-test.cc
+++ b/src/kudu/util/net/net_util-test.cc
@@ -26,7 +26,6 @@
 #include "kudu/util/net/net_util.h"
 #include "kudu/util/net/socket.h"
 #include "kudu/util/net/sockaddr.h"
-#include "kudu/util/net/ssl_socket.h"
 #include "kudu/util/status.h"
 #include "kudu/util/test_util.h"
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/71638f5d/src/kudu/util/net/ssl_factory.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/net/ssl_factory.cc b/src/kudu/util/net/ssl_factory.cc
deleted file mode 100644
index b8016a4..0000000
--- a/src/kudu/util/net/ssl_factory.cc
+++ /dev/null
@@ -1,153 +0,0 @@
-// 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 <mutex>
-#include <vector>
-
-#include <openssl/err.h>
-#include <openssl/rand.h>
-#include <openssl/ssl.h>
-#include <openssl/x509v3.h>
-
-#include "kudu/util/debug/leakcheck_disabler.h"
-#include "kudu/util/mutex.h"
-#include "kudu/util/thread.h"
-#include "kudu/util/net/ssl_factory.h"
-#include "kudu/util/net/ssl_socket.h"
-
-namespace kudu {
-
-// These non-POD elements will be alive for the lifetime of the process, so don't allocate in
-// static storage.
-static Mutex* g_ssl_mutexes;
-
-// Lock/Unlock the nth lock. Only to be used by OpenSSL.
-static void CryptoLockingCallback(int mode, int n, const char* /*unused*/, int /*unused*/) {
-  if (mode & CRYPTO_LOCK) {
-    g_ssl_mutexes[n].Acquire();
-  } else {
-    g_ssl_mutexes[n].Release();
-  }
-}
-
-// Return the current pthread's tid. Only to be used by OpenSSL.
-static void CryptoThreadIDCallback(CRYPTO_THREADID* id) {
-  return CRYPTO_THREADID_set_numeric(id, Thread::UniqueThreadId());
-}
-
-void DoSSLInit() {
-  SSL_library_init();
-  SSL_load_error_strings();
-  OpenSSL_add_all_algorithms();
-  RAND_poll();
-
-  debug::ScopedLeakCheckDisabler d;
-  g_ssl_mutexes = new Mutex[CRYPTO_num_locks()];
-
-  // Callbacks used by OpenSSL required in a multi-threaded setting.
-  CRYPTO_set_locking_callback(CryptoLockingCallback);
-  CRYPTO_THREADID_set_callback(CryptoThreadIDCallback);
-}
-
-SSLFactory::SSLFactory() : ctx_(nullptr, SSL_CTX_free) {
-  static std::once_flag ssl_once;
-  std::call_once(ssl_once, DoSSLInit);
-}
-
-SSLFactory::~SSLFactory() {
-}
-
-Status SSLFactory::Init() {
-  CHECK(!ctx_.get());
-  // NOTE: 'SSLv23 method' sounds like it would enable only SSLv2 and SSLv3, but in fact
-  // this is a sort of wildcard which enables all methods (including TLSv1 and later).
-  // We explicitly disable SSLv2 and SSLv3 below so that only TLS methods remain.
-  // See the discussion on https://trac.torproject.org/projects/tor/ticket/11598 for more
-  // info.
-  ctx_.reset(SSL_CTX_new(SSLv23_method()));
-  if (!ctx_) {
-    return Status::RuntimeError("Could not create SSL context");
-  }
-  SSL_CTX_set_mode(ctx_.get(), SSL_MODE_AUTO_RETRY);
-
-  // Disable SSLv2 and SSLv3 which are vulnerable to various issues such as POODLE.
-  // We support versions back to TLSv1.0 since OpenSSL on RHEL 6.4 and earlier does not
-  // not support TLSv1.1 or later.
-  //
-  // Disable SSL/TLS compression to free up CPU resources and be less prone
-  // to attacks exploiting the compression feature:
-  //   https://tools.ietf.org/html/rfc7525#section-3.3
-  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);
-  return Status::OK();
-}
-
-std::string SSLFactory::GetLastError(int errno_copy) {
-  int error_code = ERR_get_error();
-  if (error_code == 0) return kudu::ErrnoToString(errno_copy);
-  const char* error_reason = ERR_reason_error_string(error_code);
-  if (error_reason != NULL) return error_reason;
-  return strings::Substitute("SSL error $0", error_code);
-}
-
-Status SSLFactory::LoadCertificate(const std::string& certificate_path) {
-  ERR_clear_error();
-  errno = 0;
-  if (SSL_CTX_use_certificate_file(ctx_.get(), certificate_path.c_str(), SSL_FILETYPE_PEM) != 1) {
-    return Status::NotFound(
-        "Failed to load certificate file '" + certificate_path + "': " + GetLastError(errno));
-  }
-  return Status::OK();
-}
-
-Status SSLFactory::LoadPrivateKey(const std::string& key_path) {
-  ERR_clear_error();
-  errno = 0;
-  if (SSL_CTX_use_PrivateKey_file(ctx_.get(), key_path.c_str(), SSL_FILETYPE_PEM) != 1) {
-    return Status::NotFound(
-        "Failed to load private key file '" + key_path + "': " + GetLastError(errno));
-  }
-  return Status::OK();
-}
-
-Status SSLFactory::LoadCertificateAuthority(const std::string& certificate_path) {
-  ERR_clear_error();
-  errno = 0;
-  if (SSL_CTX_load_verify_locations(ctx_.get(), certificate_path.c_str(), nullptr) != 1) {
-    return Status::NotFound(
-        "Failed to load certificate authority file '" + certificate_path + "': " +
-            GetLastError(errno));
-  }
-  return Status::OK();
-}
-
-std::unique_ptr<SSLSocket> SSLFactory::CreateSocket(int socket_fd, bool is_server) {
-  CHECK(ctx_);
-  // Create SSL object and transfer ownership to the SSLSocket object created.
-  SSL* ssl = SSL_new(ctx_.get());
-  if (ssl == nullptr) {
-    return nullptr;
-  }
-  std::unique_ptr<SSLSocket> socket(new SSLSocket(socket_fd, ssl, is_server));
-  return socket;
-  //return new SSLSocket(socket_fd, ssl, is_server);
-}
-
-} // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/71638f5d/src/kudu/util/net/ssl_factory.h
----------------------------------------------------------------------
diff --git a/src/kudu/util/net/ssl_factory.h b/src/kudu/util/net/ssl_factory.h
deleted file mode 100644
index 8fb0147..0000000
--- a/src/kudu/util/net/ssl_factory.h
+++ /dev/null
@@ -1,66 +0,0 @@
-// 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 <memory>
-
-#include "kudu/gutil/macros.h"
-#include "kudu/gutil/strings/substitute.h"
-#include "kudu/util/errno.h"
-#include "kudu/util/status.h"
-
-struct ssl_ctx_st;
-typedef ssl_ctx_st SSL_CTX;
-
-namespace kudu {
-
-class Sockaddr;
-class SSLSocket;
-
-class SSLFactory {
- public:
-  SSLFactory();
-
-  ~SSLFactory();
-
-  // Set up the SSL_CTX and choose encryption preferences.
-  Status Init();
-
-  // Load the server certificate.
-  Status LoadCertificate(const std::string& certificate_path);
-
-  // Load the private key for the server certificate.
-  Status LoadPrivateKey(const std::string& key_path);
-
-  // Load the certificate authority.
-  Status LoadCertificateAuthority(const std::string& certificate_path);
-
-  // Create an SSLSocket wrapped around the file descriptor 'socket_fd'. 'is_server' denotes if it's
-  // a server socket. The 'socket_fd' is closed when this object is destroyed.
-  std::unique_ptr<SSLSocket> CreateSocket(int socket_fd, bool is_server);
-
- private:
-  friend class SSLSocket;
-  std::unique_ptr<SSL_CTX, std::function<void(SSL_CTX*)>> ctx_;
-
-  // Gets the last error from the thread local SSL error queue. If no error exists, it returns
-  // the error corresponding to 'errno_copy'.
-  static std::string GetLastError(int errno_copy);
-};
-
-} // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/71638f5d/src/kudu/util/net/ssl_socket.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/net/ssl_socket.cc b/src/kudu/util/net/ssl_socket.cc
deleted file mode 100644
index 3382521..0000000
--- a/src/kudu/util/net/ssl_socket.cc
+++ /dev/null
@@ -1,174 +0,0 @@
-// 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 <errno.h>
-
-#include <vector>
-
-#include <openssl/err.h>
-#include <openssl/ssl.h>
-#include <openssl/x509.h>
-#include <openssl/x509v3.h>
-
-#include "kudu/gutil/strings/util.h"
-#include "kudu/gutil/strings/split.h"
-#include "kudu/util/errno.h"
-#include "kudu/util/net/ssl_factory.h"
-#include "kudu/util/net/ssl_socket.h"
-#include "kudu/util/net/sockaddr.h"
-
-#if OPENSSL_VERSION_NUMBER < 0x10002000L
-#include "kudu/util/x509_check_host.h"
-#endif // OPENSSL_VERSION_NUMBER
-
-namespace kudu {
-
-SSLSocket::SSLSocket(int fd, SSL* ssl, bool is_server) :
-    Socket(fd), ssl_(ssl), is_server_(is_server) {
-  SSL_set_fd(ssl_, fd);
-}
-
-SSLSocket::~SSLSocket() {
-}
-
-Status SSLSocket::DoHandshake() {
-  CHECK(ssl_);
-  ERR_clear_error();
-  errno = 0;
-  int ret;
-  if (is_server_) {
-    ret = SSL_accept(ssl_);
-  } else {
-    ret = SSL_connect(ssl_);
-  }
-  if (ret <= 0) return Status::NetworkError(SSLFactory::GetLastError(errno));
-
-  // Verify if the handshake was successful.
-  int rc = SSL_get_verify_result(ssl_);
-  if (rc != X509_V_OK) {
-    return Status::NetworkError("SSL_get_verify_result()", X509_verify_cert_error_string(rc));
-  }
-
-  // Get the peer certificate.
-  std::unique_ptr<X509, void(*)(X509*)> cert(
-      SSL_get_peer_certificate(ssl_), [](X509* x) { X509_free(x); });
-  cert.reset(SSL_get_peer_certificate(ssl_));
-  if (cert == nullptr) {
-    if (SSL_get_verify_mode(ssl_) & SSL_VERIFY_FAIL_IF_NO_PEER_CERT) {
-      return Status::NetworkError("Handshake failed: Could not retreive peer certificate");
-    }
-  }
-
-  // Get the peer's hostname
-  Sockaddr peer_addr;
-  if (!GetPeerAddress(&peer_addr).ok()) {
-    return Status::NetworkError("Handshake failed: Could not retrieve peer address");
-  }
-  std::string peer_hostname;
-  RETURN_NOT_OK(peer_addr.LookupHostname(&peer_hostname));
-
-  // Check if the hostname matches with either the Common Name or any of the Subject Alternative
-  // Names of the certificate.
-  int match;
-  if ((match = X509_check_host(
-      cert.get(), peer_hostname.c_str(), peer_hostname.length(), 0, nullptr)) == 0) {
-    return Status::NetworkError("Handshake failed: Could not verify host with certificate");
-  }
-  if (match < 0) {
-    return Status::NetworkError("Handshake failed:", SSLFactory::GetLastError(errno));
-  }
-  CHECK(match == 1);
-  return Status::OK();
-}
-
-Status SSLSocket::Write(const uint8_t *buf, int32_t amt, int32_t *nwritten) {
-  CHECK(ssl_);
-  ERR_clear_error();
-  errno = 0;
-  int32_t bytes_written = SSL_write(ssl_, buf, amt);
-  if (bytes_written <= 0) {
-    if (SSL_get_error(ssl_, bytes_written) == SSL_ERROR_WANT_WRITE) {
-      // Socket not ready to write yet.
-      *nwritten = 0;
-      return Status::OK();
-    }
-    return Status::NetworkError("SSL_write", SSLFactory::GetLastError(errno));
-  }
-  *nwritten = bytes_written;
-  return Status::OK();
-}
-
-Status SSLSocket::Writev(const struct ::iovec *iov, int iov_len,
-                      int32_t *nwritten) {
-  CHECK(ssl_);
-  ERR_clear_error();
-  int32_t total_written = 0;
-  // Allows packets to be aggresively be accumulated before sending.
-  RETURN_NOT_OK(SetTcpCork(1));
-  Status write_status = Status::OK();
-  for (int i = 0; i < iov_len; ++i) {
-    int32_t frame_size = iov[i].iov_len;
-    // Don't return before unsetting TCP_CORK.
-    write_status = Write(static_cast<uint8_t*>(iov[i].iov_base), frame_size, nwritten);
-    total_written += *nwritten;
-    if (*nwritten < frame_size) break;
-  }
-  RETURN_NOT_OK(SetTcpCork(0));
-  *nwritten = total_written;
-  return write_status;
-}
-
-Status SSLSocket::Recv(uint8_t *buf, int32_t amt, int32_t *nread) {
-  CHECK(ssl_);
-  ERR_clear_error();
-  errno = 0;
-  int32_t bytes_read = SSL_read(ssl_, buf, amt);
-  if (bytes_read <= 0) {
-    if (bytes_read == 0 && SSL_get_shutdown(ssl_) == SSL_RECEIVED_SHUTDOWN) {
-      return Status::NetworkError("SSLSocket::Recv() for EOF from remote", Slice(), ESHUTDOWN);
-    }
-    if (SSL_get_error(ssl_, bytes_read) == SSL_ERROR_WANT_READ) {
-      // Nothing available to read yet.
-      *nread = 0;
-      return Status::OK();
-    }
-    return Status::NetworkError("SSL_read", SSLFactory::GetLastError(errno));
-  }
-  *nread = bytes_read;
-  return Status::OK();
-}
-
-Status SSLSocket::Close() {
-  CHECK(ssl_);
-  ERR_clear_error();
-  errno = 0;
-  int32_t ret = SSL_shutdown(ssl_);
-  Status shutdown_status;
-  if (ret < 0 && errno != EAGAIN) {
-    // We still need to close the underlying socket, so don't return just yet.
-    shutdown_status = Status::NetworkError("SSL_Shutdown", SSLFactory::GetLastError(errno));
-  }
-  SSL_free(ssl_);
-  ssl_ = nullptr;
-  ERR_remove_state(0);
-
-  Status close_status = Socket::Close();
-  if (!close_status.ok()) return close_status.CloneAndPrepend(shutdown_status.message());
-  return shutdown_status;
-}
-
-} // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/71638f5d/src/kudu/util/net/ssl_socket.h
----------------------------------------------------------------------
diff --git a/src/kudu/util/net/ssl_socket.h b/src/kudu/util/net/ssl_socket.h
deleted file mode 100644
index fd67cfe..0000000
--- a/src/kudu/util/net/ssl_socket.h
+++ /dev/null
@@ -1,60 +0,0 @@
-// 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.
-#ifndef KUDU_UTIL_NET_SSL_SOCKET_H
-#define KUDU_UTIL_NET_SSL_SOCKET_H
-
-#include <sys/uio.h>
-
-#include <string>
-
-#include "kudu/gutil/macros.h"
-#include "kudu/util/net/socket.h"
-#include "kudu/util/status.h"
-
-struct ssl_st;
-typedef ssl_st SSL;
-
-namespace kudu {
-
-class Sockaddr;
-
-class SSLSocket : public Socket {
- public:
-  SSLSocket(int fd, SSL* ssl, bool is_server);
-
-  ~SSLSocket();
-
-  // Do the SSL handshake as a client or a server and verify that the credentials were correctly
-  // verified.
-  Status DoHandshake();
-
-  Status Write(const uint8_t *buf, int32_t amt, int32_t *nwritten) override;
-
-  Status Writev(const struct ::iovec *iov, int iov_len, int32_t *nwritten) override;
-
-  Status Recv(uint8_t *buf, int32_t amt, int32_t *nread) override;
-
-  // Shutdown the connection and free the SSL state for this connection.
-  Status Close() override;
- private:
-  SSL* ssl_; // Owned.
-  bool is_server_;
-};
-
-} // namespace kudu
-
-#endif

http://git-wip-us.apache.org/repos/asf/kudu/blob/71638f5d/src/kudu/util/x509_check_host.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/x509_check_host.cc b/src/kudu/util/x509_check_host.cc
deleted file mode 100644
index d0d64c2..0000000
--- a/src/kudu/util/x509_check_host.cc
+++ /dev/null
@@ -1,441 +0,0 @@
-/*
- * Copyright 1999-2016 The OpenSSL Project Authors. All Rights Reserved.
- *
- * Licensed under the OpenSSL license (the "License").  You may not use
- * this file except in compliance with the License.  You can obtain a copy
- * in the file LICENSE in the source distribution or at
- * https://www.openssl.org/source/license.html
- */
-
-// The following is ported from the OpenSSL-1.1.0b library. The implementations
-// of the functions are for the most part the same except where mentioned in special
-// comments. Explicit casts were also added to bypass compilation errors.
-
-#include <stdio.h>
-#include <ctype.h>
-#include <string.h>
-#include <strings.h>
-
-#include <openssl/bn.h>
-#include <openssl/conf.h>
-#include <openssl/err.h>
-#include <openssl/x509v3.h>
-
-#include "kudu/util/x509_check_host.h"
-
-// Ported from include/openssl/crypto.h from OpenSSL-1.1.0b
-// Modifed to use __FILE__ and __LINE__ instead of OPENSSL_FILE and OPENSSL_LINE.
-# define OPENSSL_strndup(str, n) \
-        CRYPTO_strndup(str, n, __FILE__, __LINE__)
-
-// Ported from crypto/o_str.c from OpenSSL-1.1.0b.
-// Modified to use strnlen() instead of OPENSSL_strnlen()
-char *CRYPTO_strndup(const char *str, size_t s, const char* file, int line)
-{
-    size_t maxlen;
-    char *ret;
-
-    if (str == NULL)
-        return NULL;
-
-    maxlen = strnlen(str, s);
-
-    ret = (char*)CRYPTO_malloc(maxlen + 1, file, line);
-    if (ret) {
-        memcpy(ret, str, maxlen);
-        ret[maxlen] = '\0';
-    }
-    return ret;
-}
-
-// The remaining code is ported form crypto/x509v3/v3_utl.c
-
-typedef int (*equal_fn) (const unsigned char *pattern, size_t pattern_len,
-                         const unsigned char *subject, size_t subject_len,
-                         unsigned int flags);
-
-/* Skip pattern prefix to match "wildcard" subject */
-static void skip_prefix(const unsigned char **p, size_t *plen,
-                        size_t subject_len,
-                        unsigned int flags)
-{
-    const unsigned char *pattern = *p;
-    size_t pattern_len = *plen;
-
-    /*
-     * If subject starts with a leading '.' followed by more octets, and
-     * pattern is longer, compare just an equal-length suffix with the
-     * full subject (starting at the '.'), provided the prefix contains
-     * no NULs.
-     */
-    if ((flags & _X509_CHECK_FLAG_DOT_SUBDOMAINS) == 0)
-        return;
-
-    while (pattern_len > subject_len && *pattern) {
-        if ((flags & X509_CHECK_FLAG_SINGLE_LABEL_SUBDOMAINS) &&
-            *pattern == '.')
-            break;
-        ++pattern;
-        --pattern_len;
-    }
-
-    /* Skip if entire prefix acceptable */
-    if (pattern_len == subject_len) {
-        *p = pattern;
-        *plen = pattern_len;
-    }
-}
-
-/* Compare while ASCII ignoring case. */
-static int equal_nocase(const unsigned char *pattern, size_t pattern_len,
-                        const unsigned char *subject, size_t subject_len,
-                        unsigned int flags)
-{
-    skip_prefix(&pattern, &pattern_len, subject_len, flags);
-    if (pattern_len != subject_len)
-        return 0;
-    while (pattern_len) {
-        unsigned char l = *pattern;
-        unsigned char r = *subject;
-        /* The pattern must not contain NUL characters. */
-        if (l == 0)
-            return 0;
-        if (l != r) {
-            if ('A' <= l && l <= 'Z')
-                l = (l - 'A') + 'a';
-            if ('A' <= r && r <= 'Z')
-                r = (r - 'A') + 'a';
-            if (l != r)
-                return 0;
-        }
-        ++pattern;
-        ++subject;
-        --pattern_len;
-    }
-    return 1;
-}
-
-/* Compare using memcmp. */
-static int equal_case(const unsigned char *pattern, size_t pattern_len,
-                      const unsigned char *subject, size_t subject_len,
-                      unsigned int flags)
-{
-    skip_prefix(&pattern, &pattern_len, subject_len, flags);
-    if (pattern_len != subject_len)
-        return 0;
-    return !memcmp(pattern, subject, pattern_len);
-}
-
-/*
- * RFC 5280, section 7.5, requires that only the domain is compared in a
- * case-insensitive manner.
- */
-static int equal_email(const unsigned char *a, size_t a_len,
-                       const unsigned char *b, size_t b_len,
-                       unsigned int unused_flags)
-{
-    size_t i = a_len;
-    if (a_len != b_len)
-        return 0;
-    /*
-     * We search backwards for the '@' character, so that we do not have to
-     * deal with quoted local-parts.  The domain part is compared in a
-     * case-insensitive manner.
-     */
-    while (i > 0) {
-        --i;
-        if (a[i] == '@' || b[i] == '@') {
-            if (!equal_nocase(a + i, a_len - i, b + i, a_len - i, 0))
-                return 0;
-            break;
-        }
-    }
-    if (i == 0)
-        i = a_len;
-    return equal_case(a, i, b, i, 0);
-}
-
-/*
- * Compare the prefix and suffix with the subject, and check that the
- * characters in-between are valid.
- */
-static int wildcard_match(const unsigned char *prefix, size_t prefix_len,
-                          const unsigned char *suffix, size_t suffix_len,
-                          const unsigned char *subject, size_t subject_len,
-                          unsigned int flags)
-{
-    const unsigned char *wildcard_start;
-    const unsigned char *wildcard_end;
-    const unsigned char *p;
-    int allow_multi = 0;
-    int allow_idna = 0;
-
-    if (subject_len < prefix_len + suffix_len)
-        return 0;
-    if (!equal_nocase(prefix, prefix_len, subject, prefix_len, flags))
-        return 0;
-    wildcard_start = subject + prefix_len;
-    wildcard_end = subject + (subject_len - suffix_len);
-    if (!equal_nocase(wildcard_end, suffix_len, suffix, suffix_len, flags))
-        return 0;
-    /*
-     * If the wildcard makes up the entire first label, it must match at
-     * least one character.
-     */
-    if (prefix_len == 0 && *suffix == '.') {
-        if (wildcard_start == wildcard_end)
-            return 0;
-        allow_idna = 1;
-        if (flags & X509_CHECK_FLAG_MULTI_LABEL_WILDCARDS)
-            allow_multi = 1;
-    }
-    /* IDNA labels cannot match partial wildcards */
-    if (!allow_idna &&
-        subject_len >= 4 && strncasecmp((char *)subject, "xn--", 4) == 0)
-        return 0;
-    /* The wildcard may match a literal '*' */
-    if (wildcard_end == wildcard_start + 1 && *wildcard_start == '*')
-        return 1;
-    /*
-     * Check that the part matched by the wildcard contains only
-     * permitted characters and only matches a single label unless
-     * allow_multi is set.
-     */
-    for (p = wildcard_start; p != wildcard_end; ++p)
-        if (!(('0' <= *p && *p <= '9') ||
-              ('A' <= *p && *p <= 'Z') ||
-              ('a' <= *p && *p <= 'z') ||
-              *p == '-' || (allow_multi && *p == '.')))
-            return 0;
-    return 1;
-}
-
-#define LABEL_START     (1 << 0)
-#define LABEL_END       (1 << 1)
-#define LABEL_HYPHEN    (1 << 2)
-#define LABEL_IDNA      (1 << 3)
-
-static const unsigned char *valid_star(const unsigned char *p, size_t len,
-                                       unsigned int flags)
-{
-    const unsigned char *star = 0;
-    size_t i;
-    int state = LABEL_START;
-    int dots = 0;
-    for (i = 0; i < len; ++i) {
-        /*
-         * Locate first and only legal wildcard, either at the start
-         * or end of a non-IDNA first and not final label.
-         */
-        if (p[i] == '*') {
-            int atstart = (state & LABEL_START);
-            int atend = (i == len - 1 || p[i + 1] == '.');
-            /*-
-             * At most one wildcard per pattern.
-             * No wildcards in IDNA labels.
-             * No wildcards after the first label.
-             */
-            if (star != NULL || (state & LABEL_IDNA) != 0 || dots)
-                return NULL;
-            /* Only full-label '*.example.com' wildcards? */
-            if ((flags & X509_CHECK_FLAG_NO_PARTIAL_WILDCARDS)
-                && (!atstart || !atend))
-                return NULL;
-            /* No 'foo*bar' wildcards */
-            if (!atstart && !atend)
-                return NULL;
-            star = &p[i];
-            state &= ~LABEL_START;
-        } else if (('a' <= p[i] && p[i] <= 'z')
-                   || ('A' <= p[i] && p[i] <= 'Z')
-                   || ('0' <= p[i] && p[i] <= '9')) {
-            if ((state & LABEL_START) != 0
-                && len - i >= 4 && strncasecmp((char *)&p[i], "xn--", 4) == 0)
-                state |= LABEL_IDNA;
-            state &= ~(LABEL_HYPHEN | LABEL_START);
-        } else if (p[i] == '.') {
-            if ((state & (LABEL_HYPHEN | LABEL_START)) != 0)
-                return NULL;
-            state = LABEL_START;
-            ++dots;
-        } else if (p[i] == '-') {
-            /* no domain/subdomain starts with '-' */
-            if ((state & LABEL_START) != 0)
-                return NULL;
-            state |= LABEL_HYPHEN;
-        } else
-            return NULL;
-    }
-
-    /*
-     * The final label must not end in a hyphen or ".", and
-     * there must be at least two dots after the star.
-     */
-    if ((state & (LABEL_START | LABEL_HYPHEN)) != 0 || dots < 2)
-        return NULL;
-    return star;
-}
-
-/* Compare using wildcards. */
-static int equal_wildcard(const unsigned char *pattern, size_t pattern_len,
-                          const unsigned char *subject, size_t subject_len,
-                          unsigned int flags)
-{
-    const unsigned char *star = NULL;
-
-    /*
-     * Subject names starting with '.' can only match a wildcard pattern
-     * via a subject sub-domain pattern suffix match.
-     */
-    if (!(subject_len > 1 && subject[0] == '.'))
-        star = valid_star(pattern, pattern_len, flags);
-    if (star == NULL)
-        return equal_nocase(pattern, pattern_len,
-                            subject, subject_len, flags);
-    return wildcard_match(pattern, star - pattern,
-                          star + 1, (pattern + pattern_len) - star - 1,
-                          subject, subject_len, flags);
-}
-
-/*
- * Compare an ASN1_STRING to a supplied string. If they match return 1. If
- * cmp_type > 0 only compare if string matches the type, otherwise convert it
- * to UTF8.
- */
-
-static int do_check_string(const ASN1_STRING *a, int cmp_type, equal_fn equal,
-                           unsigned int flags, const char *b, size_t blen,
-                           char **peername)
-{
-    int rv = 0;
-
-    if (!a->data || !a->length)
-        return 0;
-    if (cmp_type > 0) {
-        if (cmp_type != a->type)
-            return 0;
-        if (cmp_type == V_ASN1_IA5STRING)
-            rv = equal(a->data, a->length, (unsigned char *)b, blen, flags);
-        else if (a->length == (int)blen && !memcmp(a->data, b, blen))
-            rv = 1;
-        if (rv > 0 && peername)
-            *peername = OPENSSL_strndup((char *)a->data, a->length);
-    } else {
-        int astrlen;
-        unsigned char *astr;
-        astrlen = ASN1_STRING_to_UTF8(&astr, (ASN1_STRING*)a);
-        if (astrlen < 0) {
-            /*
-             * -1 could be an internal malloc failure or a decoding error from
-             * malformed input; we can't distinguish.
-             */
-            return -1;
-        }
-        rv = equal(astr, astrlen, (unsigned char *)b, blen, flags);
-        if (rv > 0 && peername)
-            *peername = OPENSSL_strndup((char *)astr, astrlen);
-            //*peername = strndup((char *)astr, astrlen);
-        OPENSSL_free(astr);
-    }
-    return rv;
-}
-
-static int do_x509_check(X509 *x, const char *chk, size_t chklen,
-                         unsigned int flags, int check_type, char **peername)
-{
-    GENERAL_NAMES *gens = NULL;
-    X509_NAME *name = NULL;
-    int i;
-    int cnid = NID_undef;
-    int alt_type;
-    int san_present = 0;
-    int rv = 0;
-    equal_fn equal;
-
-    /* See below, this flag is internal-only */
-    flags &= ~_X509_CHECK_FLAG_DOT_SUBDOMAINS;
-    if (check_type == GEN_EMAIL) {
-        cnid = NID_pkcs9_emailAddress;
-        alt_type = V_ASN1_IA5STRING;
-        equal = equal_email;
-    } else if (check_type == GEN_DNS) {
-        cnid = NID_commonName;
-        /* Implicit client-side DNS sub-domain pattern */
-        if (chklen > 1 && chk[0] == '.')
-            flags |= _X509_CHECK_FLAG_DOT_SUBDOMAINS;
-        alt_type = V_ASN1_IA5STRING;
-        if (flags & X509_CHECK_FLAG_NO_WILDCARDS)
-            equal = equal_nocase;
-        else
-            equal = equal_wildcard;
-    } else {
-        alt_type = V_ASN1_OCTET_STRING;
-        equal = equal_case;
-    }
-
-    if (chklen == 0)
-        chklen = strlen(chk);
-
-    gens = (GENERAL_NAMES*)X509_get_ext_d2i(x, NID_subject_alt_name, NULL, NULL);
-    if (gens) {
-        for (i = 0; i < sk_GENERAL_NAME_num(gens); i++) {
-            GENERAL_NAME *gen;
-            ASN1_STRING *cstr;
-            gen = sk_GENERAL_NAME_value(gens, i);
-            if (gen->type != check_type)
-                continue;
-            san_present = 1;
-            if (check_type == GEN_EMAIL)
-                cstr = gen->d.rfc822Name;
-            else if (check_type == GEN_DNS)
-                cstr = gen->d.dNSName;
-            else
-                cstr = gen->d.iPAddress;
-            /* Positive on success, negative on error! */
-            if ((rv = do_check_string(cstr, alt_type, equal, flags,
-                                      chk, chklen, peername)) != 0)
-                break;
-        }
-        GENERAL_NAMES_free(gens);
-        if (rv != 0)
-            return rv;
-        if (san_present && !(flags & X509_CHECK_FLAG_ALWAYS_CHECK_SUBJECT))
-            return 0;
-    }
-
-    /* We're done if CN-ID is not pertinent */
-    if (cnid == NID_undef || (flags & X509_CHECK_FLAG_NEVER_CHECK_SUBJECT))
-        return 0;
-
-    i = -1;
-    name = X509_get_subject_name(x);
-    while ((i = X509_NAME_get_index_by_NID(name, cnid, i)) >= 0) {
-        const X509_NAME_ENTRY *ne = X509_NAME_get_entry(name, i);
-        const ASN1_STRING *str = X509_NAME_ENTRY_get_data((X509_NAME_ENTRY*)ne);
-
-        /* Positive on success, negative on error! */
-        if ((rv = do_check_string(str, -1, equal, flags,
-                                  chk, chklen, peername)) != 0)
-            return rv;
-    }
-    return 0;
-}
-
-int X509_check_host(X509 *x, const char *chk, size_t chklen,
-                    unsigned int flags, char **peername)
-{
-    if (chk == NULL)
-        return -2;
-    /*
-     * Embedded NULs are disallowed, except as the last character of a
-     * string of length 2 or more (tolerate caller including terminating
-     * NUL in string length).
-     */
-    if (chklen == 0)
-        chklen = strlen(chk);
-    else if (memchr(chk, '\0', chklen > 1 ? chklen - 1 : chklen))
-        return -2;
-    if (chklen > 1 && chk[chklen - 1] == '\0')
-        --chklen;
-    return do_x509_check(x, chk, chklen, flags, GEN_DNS, peername);
-}

http://git-wip-us.apache.org/repos/asf/kudu/blob/71638f5d/src/kudu/util/x509_check_host.h
----------------------------------------------------------------------
diff --git a/src/kudu/util/x509_check_host.h b/src/kudu/util/x509_check_host.h
deleted file mode 100644
index cc1b6cf..0000000
--- a/src/kudu/util/x509_check_host.h
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * Copyright 1999-2016 The OpenSSL Project Authors. All Rights Reserved.
- *
- * Licensed under the OpenSSL license (the "License").  You may not use
- * this file except in compliance with the License.  You can obtain a copy
- * in the file LICENSE in the source distribution or at
- * https://www.openssl.org/source/license.html
- */
-
-// The following is ported from the OpenSSL-1.1.0b library.
-
-#ifndef X509_CHECK_HOST_H
-# define X509_CHECK_HOST_H
-
-#include <stdlib.h>
-
-#include <openssl/x509.h>
-
-/* Flags for X509_check_* functions */
-
-/*
- * Always check subject name for host match even if subject alt names present
- */
-# define X509_CHECK_FLAG_ALWAYS_CHECK_SUBJECT    0x1
-/* Disable wildcard matching for dnsName fields and common name. */
-# define X509_CHECK_FLAG_NO_WILDCARDS    0x2
-/* Wildcards must not match a partial label. */
-# define X509_CHECK_FLAG_NO_PARTIAL_WILDCARDS 0x4
-/* Allow (non-partial) wildcards to match multiple labels. */
-# define X509_CHECK_FLAG_MULTI_LABEL_WILDCARDS 0x8
-/* Constraint verifier subdomain patterns to match a single labels. */
-# define X509_CHECK_FLAG_SINGLE_LABEL_SUBDOMAINS 0x10
-/* Never check the subject CN */
-# define X509_CHECK_FLAG_NEVER_CHECK_SUBJECT    0x20
-/*
- * Match reference identifiers starting with "." to any sub-domain.
- * This is a non-public flag, turned on implicitly when the subject
- * reference identity is a DNS name.
- */
-# define _X509_CHECK_FLAG_DOT_SUBDOMAINS 0x8000
-
-// Checks if the certificate Subject Alternative Name (SAN) or Subject CommonName (CN)
-// matches the specified host name, which must be encoded in the preferred name syntax
-// described in section 3.5 of RFC 1034.
-int X509_check_host(X509 *x, const char *chk, size_t chklen,
-                    unsigned int flags, char **peername);
-
-#endif // X509_CHECK_HOST_H


[2/2] kudu git commit: Add openssl_util

Posted by da...@apache.org.
Add openssl_util

Also migrates the existing ssl_factory static initialization to use the
new openssl_util version.

Change-Id: I1fb8bc2fb4153eff7adf896e43a979783de6f4b8
Reviewed-on: http://gerrit.cloudera.org:8080/5704
Reviewed-by: Todd Lipcon <to...@apache.org>
Reviewed-by: Alexey Serbin <as...@cloudera.com>
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/f907db13
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/f907db13
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/f907db13

Branch: refs/heads/master
Commit: f907db137c05da5e10353fb53f5a431efdac62c0
Parents: 71638f5
Author: Dan Burkert <da...@apache.org>
Authored: Thu Jan 12 16:04:39 2017 -0800
Committer: Dan Burkert <da...@apache.org>
Committed: Fri Jan 13 03:00:43 2017 +0000

----------------------------------------------------------------------
 src/kudu/security/CMakeLists.txt  |   1 +
 src/kudu/security/openssl_util.cc | 111 +++++++++++++++++++++++++++++++++
 src/kudu/security/openssl_util.h  |  39 ++++++++++++
 src/kudu/security/ssl_factory.cc  |  37 +----------
 src/kudu/security/ssl_factory.h   |   4 +-
 src/kudu/security/ssl_socket.h    |   4 +-
 6 files changed, 155 insertions(+), 41 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/f907db13/src/kudu/security/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/kudu/security/CMakeLists.txt b/src/kudu/security/CMakeLists.txt
index 2577771..2e0b3ee 100644
--- a/src/kudu/security/CMakeLists.txt
+++ b/src/kudu/security/CMakeLists.txt
@@ -31,6 +31,7 @@ endif()
 
 set(SECURITY_SRCS
   init.cc
+  openssl_util.cc
   ${PORTED_X509_CHECK_HOST_CC}
   ssl_factory.cc
   ssl_socket.cc)

http://git-wip-us.apache.org/repos/asf/kudu/blob/f907db13/src/kudu/security/openssl_util.cc
----------------------------------------------------------------------
diff --git a/src/kudu/security/openssl_util.cc b/src/kudu/security/openssl_util.cc
new file mode 100644
index 0000000..11d9b5c
--- /dev/null
+++ b/src/kudu/security/openssl_util.cc
@@ -0,0 +1,111 @@
+// 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/openssl_util.h"
+
+#include <mutex>
+#include <sstream>
+#include <string>
+#include <vector>
+
+#include <openssl/err.h>
+#include <openssl/rand.h>
+#include <openssl/ssl.h>
+
+#include "kudu/util/debug/leakcheck_disabler.h"
+#include "kudu/util/mutex.h"
+#include "kudu/util/thread.h"
+
+using std::ostringstream;
+using std::string;
+using std::vector;
+
+namespace kudu {
+namespace security {
+
+namespace {
+
+vector<Mutex*> kCryptoLocks;
+
+// Lock/Unlock the nth lock. Only to be used by OpenSSL.
+void LockingCB(int mode, int type, const char* /*file*/, int /*line*/) {
+  DCHECK(!kCryptoLocks.empty());
+  Mutex* m = kCryptoLocks[type];
+  DCHECK(m);
+  if (mode & CRYPTO_LOCK) {
+    m->lock();
+  } else {
+    m->unlock();
+  }
+}
+
+// Return the current pthread's tid. Only to be used by OpenSSL.
+void ThreadIdCB(CRYPTO_THREADID* tid) {
+  CRYPTO_THREADID_set_numeric(tid, Thread::UniqueThreadId());
+}
+
+void DoInitializeOpenSSL() {
+  SSL_library_init();
+  SSL_load_error_strings();
+  OpenSSL_add_all_algorithms();
+  RAND_poll();
+
+  // Initialize the OpenSSL mutexes. We intentionally leak these, so ignore
+  // LSAN warnings.
+  debug::ScopedLeakCheckDisabler d;
+  int num_locks = CRYPTO_num_locks();
+  kCryptoLocks.reserve(num_locks);
+  for (int i = 0; i < num_locks; i++) {
+    kCryptoLocks.emplace_back(new Mutex());
+  }
+
+  // Callbacks used by OpenSSL required in a multi-threaded setting.
+  CRYPTO_set_locking_callback(LockingCB);
+  CRYPTO_THREADID_set_callback(ThreadIdCB);
+}
+} // namespace
+
+void InitializeOpenSSL() {
+  static std::once_flag ssl_once;
+  std::call_once(ssl_once, DoInitializeOpenSSL);
+}
+
+string GetOpenSSLErrors() {
+  ostringstream serr;
+  uint32_t l;
+  int line, flags;
+  const char *file, *data;
+  bool is_first = true;
+  while ((l = ERR_get_error_line_data(&file, &line, &data, &flags)) != 0) {
+    if (is_first) {
+      is_first = false;
+    } else {
+      serr << " ";
+    }
+
+    char buf[256];
+    ERR_error_string_n(l, buf, sizeof(buf));
+    serr << buf << ":" << file << ":" << line;
+    if (flags & ERR_TXT_STRING) {
+      serr << ":" << data;
+    }
+  }
+  return serr.str();
+}
+
+} // namespace security
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/f907db13/src/kudu/security/openssl_util.h
----------------------------------------------------------------------
diff --git a/src/kudu/security/openssl_util.h b/src/kudu/security/openssl_util.h
new file mode 100644
index 0000000..1bfdb0d
--- /dev/null
+++ b/src/kudu/security/openssl_util.h
@@ -0,0 +1,39 @@
+// 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>
+
+// Forward declarations for the OpenSSL typedefs.
+typedef struct bio_st BIO;
+typedef struct ssl_ctx_st SSL_CTX;
+typedef struct ssl_st SSL;
+
+namespace kudu {
+namespace security {
+
+// Initializes static state required by the OpenSSL library.
+//
+// Safe to call multiple times.
+void InitializeOpenSSL();
+
+// Fetch the last error message from the OpenSSL library.
+std::string GetOpenSSLErrors();
+
+} // namespace security
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/f907db13/src/kudu/security/ssl_factory.cc
----------------------------------------------------------------------
diff --git a/src/kudu/security/ssl_factory.cc b/src/kudu/security/ssl_factory.cc
index 97de04b..2d011bf 100644
--- a/src/kudu/security/ssl_factory.cc
+++ b/src/kudu/security/ssl_factory.cc
@@ -21,10 +21,10 @@
 #include <vector>
 
 #include <openssl/err.h>
-#include <openssl/rand.h>
 #include <openssl/ssl.h>
 #include <openssl/x509v3.h>
 
+#include "kudu/security/openssl_util.h"
 #include "kudu/security/ssl_socket.h"
 #include "kudu/util/debug/leakcheck_disabler.h"
 #include "kudu/util/mutex.h"
@@ -32,41 +32,8 @@
 
 namespace kudu {
 
-// These non-POD elements will be alive for the lifetime of the process, so don't allocate in
-// static storage.
-static Mutex* g_ssl_mutexes;
-
-// Lock/Unlock the nth lock. Only to be used by OpenSSL.
-static void CryptoLockingCallback(int mode, int n, const char* /*unused*/, int /*unused*/) {
-  if (mode & CRYPTO_LOCK) {
-    g_ssl_mutexes[n].Acquire();
-  } else {
-    g_ssl_mutexes[n].Release();
-  }
-}
-
-// Return the current pthread's tid. Only to be used by OpenSSL.
-static void CryptoThreadIDCallback(CRYPTO_THREADID* id) {
-  return CRYPTO_THREADID_set_numeric(id, Thread::UniqueThreadId());
-}
-
-void DoSSLInit() {
-  SSL_library_init();
-  SSL_load_error_strings();
-  OpenSSL_add_all_algorithms();
-  RAND_poll();
-
-  debug::ScopedLeakCheckDisabler d;
-  g_ssl_mutexes = new Mutex[CRYPTO_num_locks()];
-
-  // Callbacks used by OpenSSL required in a multi-threaded setting.
-  CRYPTO_set_locking_callback(CryptoLockingCallback);
-  CRYPTO_THREADID_set_callback(CryptoThreadIDCallback);
-}
-
 SSLFactory::SSLFactory() : ctx_(nullptr, SSL_CTX_free) {
-  static std::once_flag ssl_once;
-  std::call_once(ssl_once, DoSSLInit);
+  security::InitializeOpenSSL();
 }
 
 SSLFactory::~SSLFactory() {

http://git-wip-us.apache.org/repos/asf/kudu/blob/f907db13/src/kudu/security/ssl_factory.h
----------------------------------------------------------------------
diff --git a/src/kudu/security/ssl_factory.h b/src/kudu/security/ssl_factory.h
index 9d2dd31..02a3afb 100644
--- a/src/kudu/security/ssl_factory.h
+++ b/src/kudu/security/ssl_factory.h
@@ -22,12 +22,10 @@
 
 #include "kudu/gutil/macros.h"
 #include "kudu/gutil/strings/substitute.h"
+#include "kudu/security/openssl_util.h"
 #include "kudu/util/errno.h"
 #include "kudu/util/status.h"
 
-struct ssl_ctx_st;
-typedef ssl_ctx_st SSL_CTX;
-
 namespace kudu {
 
 class Sockaddr;

http://git-wip-us.apache.org/repos/asf/kudu/blob/f907db13/src/kudu/security/ssl_socket.h
----------------------------------------------------------------------
diff --git a/src/kudu/security/ssl_socket.h b/src/kudu/security/ssl_socket.h
index 91fe39f..4f67d48 100644
--- a/src/kudu/security/ssl_socket.h
+++ b/src/kudu/security/ssl_socket.h
@@ -22,12 +22,10 @@
 #include <string>
 
 #include "kudu/gutil/macros.h"
+#include "kudu/security/openssl_util.h"
 #include "kudu/util/net/socket.h"
 #include "kudu/util/status.h"
 
-struct ssl_st;
-typedef ssl_st SSL;
-
 namespace kudu {
 
 class Sockaddr;