You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by ad...@apache.org on 2018/05/21 17:06:58 UTC

kudu git commit: KUDU-1889: support openssl 1.1

Repository: kudu
Updated Branches:
  refs/heads/master 9de558c2f -> 14080bb0f


KUDU-1889: support openssl 1.1

OpenSSL 1.1 is the default version available in Ubuntu 18.04.

Note that because of KUDU-2439, tests built against OpenSSL 1.1 are still
somewhat unstable. I've seen some TSAN tests occasionally crash, and some
ASAN tests (seemingly those that use the CLI to connect to a remote server)
report memory leaks.

Change-Id: If1e1c57b5563d1a4cd926b4c4a9a3c271460be04
Reviewed-on: http://gerrit.cloudera.org:8080/10436
Tested-by: Kudu Jenkins
Reviewed-by: Dan Burkert <da...@apache.org>
Reviewed-by: Alexey Serbin <as...@cloudera.com>


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

Branch: refs/heads/master
Commit: 14080bb0f49b1fb0cc95f3f13c4e893ac9992252
Parents: 9de558c
Author: Adar Dembo <ad...@cloudera.com>
Authored: Thu May 3 16:14:52 2018 -0700
Committer: Adar Dembo <ad...@cloudera.com>
Committed: Mon May 21 17:06:33 2018 +0000

----------------------------------------------------------------------
 build-support/tsan-suppressions.txt     |  9 ++++
 src/kudu/rpc/reactor.cc                 |  6 +++
 src/kudu/security/ca/cert_management.cc |  4 ++
 src/kudu/security/ca/cert_management.h  |  4 ++
 src/kudu/security/cert.cc               | 12 ++++-
 src/kudu/security/cert.h                |  6 ++-
 src/kudu/security/crypto.cc             |  4 ++
 src/kudu/security/openssl_util.cc       | 25 +++++++++++
 src/kudu/security/security-test-util.cc | 11 ++++-
 src/kudu/security/tls_context.cc        | 66 +++++++++++++++++++---------
 src/kudu/security/tls_handshake.cc      |  2 +-
 11 files changed, 124 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/14080bb0/build-support/tsan-suppressions.txt
----------------------------------------------------------------------
diff --git a/build-support/tsan-suppressions.txt b/build-support/tsan-suppressions.txt
index 4e4c5ed..f8b9a7e 100644
--- a/build-support/tsan-suppressions.txt
+++ b/build-support/tsan-suppressions.txt
@@ -99,3 +99,12 @@ race:kudu::consensus::LocalTestPeerProxy::~LocalTestPeerProxy
 # GetBoundAddresses()
 race:kudu::Webserver::GetBoundAddresses
 race:kudu::RpcServer::GetBoundAddresses
+
+# KUDU-2439: OpenSSL 1.1's atexit() handler may destroy global state while a
+# Messenger is shutting down and still accessing that state. See
+# https://github.com/openssl/openssl/issues/6214 for more details.
+#
+# This is carried out by OPENSSL_cleanup, but TSAN's unwinder doesn't
+# include any stack frame above the libcrypto lock destruction or memory release
+# call for some reason, so we have to do something more generic.
+called_from_lib:libcrypto.so

http://git-wip-us.apache.org/repos/asf/kudu/blob/14080bb0/src/kudu/rpc/reactor.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/reactor.cc b/src/kudu/rpc/reactor.cc
index d3052ea..c1832ef 100644
--- a/src/kudu/rpc/reactor.cc
+++ b/src/kudu/rpc/reactor.cc
@@ -250,7 +250,13 @@ void ReactorThread::ShutdownInternal() {
   }
 
   // Remove the OpenSSL thread state.
+  //
+  // As of OpenSSL 1.1, this [1] is a no-op and can be ignored.
+  //
+  // 1. https://www.openssl.org/docs/man1.1.0/crypto/ERR_remove_thread_state.html
+#if OPENSSL_VERSION_NUMBER < 0x10100000L
   ERR_remove_thread_state(nullptr);
+#endif
 }
 
 ReactorTask::ReactorTask() {

http://git-wip-us.apache.org/repos/asf/kudu/blob/14080bb0/src/kudu/security/ca/cert_management.cc
----------------------------------------------------------------------
diff --git a/src/kudu/security/ca/cert_management.cc b/src/kudu/security/ca/cert_management.cc
index 624958f..7ccc376 100644
--- a/src/kudu/security/ca/cert_management.cc
+++ b/src/kudu/security/ca/cert_management.cc
@@ -339,12 +339,16 @@ Status CertSigner::CopyExtensions(X509_REQ* req, X509* x) {
 Status CertSigner::FillCertTemplateFromRequest(X509_REQ* req, X509* tmpl) {
   SCOPED_OPENSSL_NO_PENDING_ERRORS;
   CHECK(req);
+
+  // As of OpenSSL 1.1, req's internals are hidden.
+#if OPENSSL_VERSION_NUMBER < 0x10100000L
   if (!req->req_info ||
       !req->req_info->pubkey ||
       !req->req_info->pubkey->public_key ||
       !req->req_info->pubkey->public_key->data) {
     return Status::RuntimeError("corrupted CSR: no public key");
   }
+#endif
   auto pub_key = ssl_make_unique(X509_REQ_get_pubkey(req));
   OPENSSL_RET_IF_NULL(pub_key, "error unpacking public key from CSR");
   const int rc = X509_REQ_verify(req, pub_key.get());

http://git-wip-us.apache.org/repos/asf/kudu/blob/14080bb0/src/kudu/security/ca/cert_management.h
----------------------------------------------------------------------
diff --git a/src/kudu/security/ca/cert_management.h b/src/kudu/security/ca/cert_management.h
index 9e43725..fb2bd0e 100644
--- a/src/kudu/security/ca/cert_management.h
+++ b/src/kudu/security/ca/cert_management.h
@@ -35,7 +35,11 @@
 // Forward declarations for the relevant OpenSSL typedefs
 // in addition to openssl_util.h.
 typedef struct asn1_string_st ASN1_INTEGER;
+#if OPENSSL_VERSION_NUMBER < 0x10100000L
 typedef struct env_md_st EVP_MD;
+#else
+typedef struct evp_md_st EVP_MD;
+#endif
 typedef struct rsa_st RSA;
 typedef struct x509_st X509;
 typedef struct X509_req_st X509_REQ;

http://git-wip-us.apache.org/repos/asf/kudu/blob/14080bb0/src/kudu/security/cert.cc
----------------------------------------------------------------------
diff --git a/src/kudu/security/cert.cc b/src/kudu/security/cert.cc
index e2fc2fb..b81d263 100644
--- a/src/kudu/security/cert.cc
+++ b/src/kudu/security/cert.cc
@@ -271,11 +271,21 @@ Status CertSignRequest::FromFile(const std::string& fpath, DataFormat format) {
 }
 
 CertSignRequest CertSignRequest::Clone() const {
+  X509_REQ* cloned_req;
+#if OPENSSL_VERSION_NUMBER < 0x10100000L
   CHECK_GT(CRYPTO_add(&data_->references, 1, CRYPTO_LOCK_X509_REQ), 1)
     << "X509_REQ use-after-free detected";
+  cloned_req = GetRawData();
+#else
+  // With OpenSSL 1.1, data structure internals are hidden, and there doesn't
+  // seem to be a public method that increments data_'s refcount.
+  cloned_req = X509_REQ_dup(GetRawData());
+  CHECK(cloned_req != nullptr)
+    << "X509 allocation failure detected: " << GetOpenSSLErrors();
+#endif
 
   CertSignRequest clone;
-  clone.AdoptRawData(GetRawData());
+  clone.AdoptRawData(cloned_req);
   return clone;
 }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/14080bb0/src/kudu/security/cert.h
----------------------------------------------------------------------
diff --git a/src/kudu/security/cert.h b/src/kudu/security/cert.h
index 9fe7a2d..4629883 100644
--- a/src/kudu/security/cert.h
+++ b/src/kudu/security/cert.h
@@ -104,7 +104,11 @@ class CertSignRequest : public RawDataWrapper<X509_REQ> {
   Status ToString(std::string* data, DataFormat format) const WARN_UNUSED_RESULT;
   Status FromFile(const std::string& fpath, DataFormat format) WARN_UNUSED_RESULT;
 
-  // Returns a shallow clone of the CSR (only a reference count is incremented).
+  // Returns a clone of the CSR.
+  //
+  // Whether this clone is deep or shallow (i.e. only a reference count is
+  // incremented) depends on the version of OpenSSL. Either way, the right
+  // thing happens when the clone goes out of scope.
   CertSignRequest Clone() const;
 
   // Returns the CSR's public key.

http://git-wip-us.apache.org/repos/asf/kudu/blob/14080bb0/src/kudu/security/crypto.cc
----------------------------------------------------------------------
diff --git a/src/kudu/security/crypto.cc b/src/kudu/security/crypto.cc
index 526dda1..234d193 100644
--- a/src/kudu/security/crypto.cc
+++ b/src/kudu/security/crypto.cc
@@ -88,7 +88,11 @@ template<> struct SslTypeTraits<RSA> {
   static constexpr auto kFreeFunc = &RSA_free;
 };
 template<> struct SslTypeTraits<EVP_MD_CTX> {
+#if OPENSSL_VERSION_NUMBER < 0x10100000L
   static constexpr auto kFreeFunc = &EVP_MD_CTX_destroy;
+#else
+  static constexpr auto kFreeFunc = &EVP_MD_CTX_free;
+#endif
 };
 
 namespace {

http://git-wip-us.apache.org/repos/asf/kudu/blob/14080bb0/src/kudu/security/openssl_util.cc
----------------------------------------------------------------------
diff --git a/src/kudu/security/openssl_util.cc b/src/kudu/security/openssl_util.cc
index 3309ca6..a32140f 100644
--- a/src/kudu/security/openssl_util.cc
+++ b/src/kudu/security/openssl_util.cc
@@ -64,6 +64,9 @@ bool g_disable_ssl_init = false;
 
 // Array of locks used by OpenSSL.
 // We use an intentionally-leaked C-style array here to avoid non-POD static data.
+//
+// As of OpenSSL 1.1, locking callbacks are no longer used.
+#if OPENSSL_VERSION_NUMBER < 0x10100000L
 Mutex* kCryptoLocks = nullptr;
 
 // Lock/Unlock the nth lock. Only to be used by OpenSSL.
@@ -76,6 +79,7 @@ void LockingCB(int mode, int type, const char* /*file*/, int /*line*/) {
     m->unlock();
   }
 }
+#endif
 
 Status CheckOpenSSLInitialized() {
   if (!CRYPTO_get_locking_callback()) {
@@ -90,6 +94,26 @@ Status CheckOpenSSLInitialized() {
 }
 
 void DoInitializeOpenSSL() {
+#if OPENSSL_VERSION_NUMBER > 0x10100000L
+  // The OPENSSL_init_ssl manpage [1] says "As of version 1.1.0 OpenSSL will
+  // automatically allocate all resources it needs so no explicit initialisation
+  // is required." However, eliding library initialization leads to a memory
+  // leak in some versions of OpenSSL 1.1 when the first OpenSSL is
+  // ERR_peek_error [2]. In Kudu this is often the
+  // case due to prolific application of SCOPED_OPENSSL_NO_PENDING_ERRORS.
+  //
+  // Rather than determine whether this particular OpenSSL instance is
+  // leak-free, we'll initialize the library explicitly.
+  //
+  // 1. https://www.openssl.org/docs/man1.1.0/ssl/OPENSSL_init_ssl.html
+  // 2. https://github.com/openssl/openssl/issues/5899
+  if (g_disable_ssl_init) {
+    VLOG(2) << "Not initializing OpenSSL (disabled by application)";
+    return;
+  }
+  CHECK_EQ(1, OPENSSL_init_ssl(0, nullptr));
+  SCOPED_OPENSSL_NO_PENDING_ERRORS;
+#else
   // In case the user's thread has left some error around, clear it.
   ERR_clear_error();
   SCOPED_OPENSSL_NO_PENDING_ERRORS;
@@ -134,6 +158,7 @@ void DoInitializeOpenSSL() {
     // Callbacks used by OpenSSL required in a multi-threaded setting.
     CRYPTO_set_locking_callback(LockingCB);
   }
+#endif
 
   g_ssl_is_initialized = true;
 }

http://git-wip-us.apache.org/repos/asf/kudu/blob/14080bb0/src/kudu/security/security-test-util.cc
----------------------------------------------------------------------
diff --git a/src/kudu/security/security-test-util.cc b/src/kudu/security/security-test-util.cc
index eca7509..40b0938 100644
--- a/src/kudu/security/security-test-util.cc
+++ b/src/kudu/security/security-test-util.cc
@@ -38,7 +38,16 @@ using ca::CertSigner;
 Status GenerateSelfSignedCAForTests(PrivateKey* ca_key, Cert* ca_cert) {
   static const int64_t kRootCaCertExpirationSeconds = 24 * 60 * 60;
   // Create a key for the self-signed CA.
-  RETURN_NOT_OK(GeneratePrivateKey(512, ca_key));
+  //
+  // OpenSSL has a concept of "security levels" which, amongst other things,
+  // place certain restrictions on key strength. OpenSSL 1.0 defaults to level
+  // 0 (no restrictions) while 1.1 defaults to level 1, which requires RSA keys
+  // to have at least 1024 bits. For simplicity, we'll just use 1024 bits here,
+  // even though shorter keys would decrease test running time.
+  //
+  // See https://www.openssl.org/docs/man1.1.0/ssl/SSL_CTX_get_security_level.html
+  // for more details.
+  RETURN_NOT_OK(GeneratePrivateKey(1024, ca_key));
 
   CaCertRequestGenerator::Config config = { "test-ca-cn" };
   RETURN_NOT_OK(CertSigner::SelfSignCA(*ca_key,

http://git-wip-us.apache.org/repos/asf/kudu/blob/14080bb0/src/kudu/security/tls_context.cc
----------------------------------------------------------------------
diff --git a/src/kudu/security/tls_context.cc b/src/kudu/security/tls_context.cc
index 3959e2c..9bf433d 100644
--- a/src/kudu/security/tls_context.cc
+++ b/src/kudu/security/tls_context.cc
@@ -90,6 +90,26 @@ template<> struct SslTypeTraits<X509_STORE_CTX> {
   static constexpr auto kFreeFunc = &X509_STORE_CTX_free;
 };
 
+namespace {
+
+Status CheckMaxSupportedTlsVersion(int tls_version, const char* tls_version_str) {
+  // OpenSSL 1.1 and newer supports all of the TLS versions we care about, so
+  // the below check is only necessary in older versions of OpenSSL.
+#if OPENSSL_VERSION_NUMBER < 0x10100000L
+  auto max_supported_tls_version = SSLv23_method()->version;
+  DCHECK_GE(max_supported_tls_version, TLS1_VERSION);
+
+  if (max_supported_tls_version < tls_version) {
+    return Status::InvalidArgument(
+        Substitute("invalid minimum TLS protocol version (--rpc_tls_min_protocol): "
+                   "this platform does not support $0", tls_version_str));
+  }
+#endif
+  return Status::OK();
+}
+
+} // anonymous namespace
+
 TlsContext::TlsContext()
     : tls_ciphers_(kudu::security::SecurityDefaults::kDefaultTlsCiphers),
       tls_min_protocol_(kudu::security::SecurityDefaults::kDefaultTlsMinVersion),
@@ -134,24 +154,11 @@ Status TlsContext::Init() {
   //   https://tools.ietf.org/html/rfc7525#section-3.3
   auto options = SSL_OP_NO_SSLv2 | SSL_OP_NO_SSLv3 | SSL_OP_NO_COMPRESSION;
 
-  auto max_supported_tls_version = SSLv23_method()->version;
-  DCHECK_GE(max_supported_tls_version, TLS1_VERSION);
-
   if (boost::iequals(tls_min_protocol_, "TLSv1.2")) {
-    if (max_supported_tls_version < TLS1_2_VERSION) {
-      return Status::InvalidArgument(
-          "invalid minimum TLS protocol version (--rpc_tls_min_protocol): "
-          "this platform does not support TLSv1.2");
-    }
-
+    RETURN_NOT_OK(CheckMaxSupportedTlsVersion(TLS1_2_VERSION, "TLSv1.2"));
     options |= SSL_OP_NO_TLSv1 | SSL_OP_NO_TLSv1_1;
   } else if (boost::iequals(tls_min_protocol_, "TLSv1.1")) {
-    if (max_supported_tls_version < TLS1_1_VERSION) {
-      return Status::InvalidArgument(
-          "invalid minimum TLS protocol version (--rpc_tls_min_protocol): "
-          "this platform does not support TLSv1.1");
-    }
-
+    RETURN_NOT_OK(CheckMaxSupportedTlsVersion(TLS1_1_VERSION, "TLSv1.1"));
     options |= SSL_OP_NO_TLSv1;
   } else if (!boost::iequals(tls_min_protocol_, "TLSv1")) {
     return Status::InvalidArgument("unknown value provided for --rpc_tls_min_protocol flag",
@@ -293,15 +300,32 @@ Status TlsContext::DumpTrustedCerts(vector<string>* cert_ders) const {
   vector<string> ret;
   auto* cert_store = SSL_CTX_get_cert_store(ctx_.get());
 
-  CRYPTO_w_lock(CRYPTO_LOCK_X509_STORE);
+#if OPENSSL_VERSION_NUMBER < 0x10100000L
+#define STORE_LOCK(CS) CRYPTO_w_lock(CRYPTO_LOCK_X509_STORE)
+#define STORE_UNLOCK(CS) CRYPTO_w_unlock(CRYPTO_LOCK_X509_STORE)
+#define STORE_GET_X509_OBJS(CS) (CS)->objs
+#define X509_OBJ_GET_TYPE(X509_OBJ) (X509_OBJ)->type
+#define X509_OBJ_GET_X509(X509_OBJ) (X509_OBJ)->data.x509
+#else
+#define STORE_LOCK(CS) CHECK_EQ(1, X509_STORE_lock(CS)) << "Could not lock certificate store"
+#define STORE_UNLOCK(CS) CHECK_EQ(1, X509_STORE_unlock(CS)) << "Could not unlock certificate store"
+#define STORE_GET_X509_OBJS(CS) X509_STORE_get0_objects(CS)
+#define X509_OBJ_GET_TYPE(X509_OBJ) X509_OBJECT_get_type(X509_OBJ)
+#define X509_OBJ_GET_X509(X509_OBJ) X509_OBJECT_get0_X509(X509_OBJ)
+#endif
+
+  STORE_LOCK(cert_store);
   auto unlock = MakeScopedCleanup([&]() {
-      CRYPTO_w_unlock(CRYPTO_LOCK_X509_STORE);
+      STORE_UNLOCK(cert_store);
     });
-  for (int i = 0; i < sk_X509_OBJECT_num(cert_store->objs); i++) {
-    X509_OBJECT* obj = sk_X509_OBJECT_value(cert_store->objs, i);
-    if (obj->type != X509_LU_X509) continue;
+  auto* objects = STORE_GET_X509_OBJS(cert_store);
+  int num_objects = sk_X509_OBJECT_num(objects);
+  for (int i = 0; i < num_objects; i++) {
+    auto* obj = sk_X509_OBJECT_value(objects, i);
+    if (X509_OBJ_GET_TYPE(obj) != X509_LU_X509) continue;
+    auto* x509 = X509_OBJ_GET_X509(obj);
     Cert c;
-    c.AdoptAndAddRefX509(obj->data.x509);
+    c.AdoptAndAddRefX509(x509);
     string der;
     RETURN_NOT_OK(c.ToString(&der, DataFormat::DER));
     ret.emplace_back(std::move(der));

http://git-wip-us.apache.org/repos/asf/kudu/blob/14080bb0/src/kudu/security/tls_handshake.cc
----------------------------------------------------------------------
diff --git a/src/kudu/security/tls_handshake.cc b/src/kudu/security/tls_handshake.cc
index 51b74fe..5a89592 100644
--- a/src/kudu/security/tls_handshake.cc
+++ b/src/kudu/security/tls_handshake.cc
@@ -89,7 +89,7 @@ Status TlsHandshake::Continue(const string& recv, string* send) {
 
   BIO* rbio = SSL_get_rbio(ssl_.get());
   int n = BIO_write(rbio, recv.data(), recv.size());
-  DCHECK_EQ(n, recv.size());
+  DCHECK(n == recv.size() || (n == -1 && recv.empty()));
   DCHECK_EQ(BIO_ctrl_pending(rbio), recv.size());
 
   int rc = SSL_do_handshake(ssl_.get());