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/03/01 19:12:28 UTC

[1/2] kudu git commit: KUDU-1901. Fix crash in concurrent OpenSSL usage

Repository: kudu
Updated Branches:
  refs/heads/master 0efc1e26a -> 0609277c4


KUDU-1901. Fix crash in concurrent OpenSSL usage

This fixes a crash in which a TLSContext's SSL_CTX was being used by a
thread to accept a connection at the same time as its underlying
certificate was being changed.

A new unit test reproduced the error pretty reliably after looping for
10-20 seconds. With the new locking, it no longer produces.

I also built OpenSSL with TSAN locally and verified that, after the
change, the only races reported within OpenSSL were those where the code
mentions the possibility of benign races.

Change-Id: I578350ba6a492e6e3ef635e9294f25fc0dc9d125
Reviewed-on: http://gerrit.cloudera.org:8080/6187
Tested-by: Kudu Jenkins
Reviewed-by: Dan Burkert <da...@apache.org>


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

Branch: refs/heads/master
Commit: f726f6ab4a94948cef8233eb45220bcf8eb4771f
Parents: 0efc1e2
Author: Todd Lipcon <to...@apache.org>
Authored: Mon Feb 27 21:46:21 2017 -0800
Committer: Todd Lipcon <to...@apache.org>
Committed: Wed Mar 1 18:57:03 2017 +0000

----------------------------------------------------------------------
 src/kudu/security/tls_context.cc        | 37 ++++++++++++--------
 src/kudu/security/tls_context.h         | 24 +++++++------
 src/kudu/security/tls_handshake-test.cc | 51 ++++++++++++++++++++++++++--
 3 files changed, 85 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/f726f6ab/src/kudu/security/tls_context.cc
----------------------------------------------------------------------
diff --git a/src/kudu/security/tls_context.cc b/src/kudu/security/tls_context.cc
index be298a4..28c8e0e 100644
--- a/src/kudu/security/tls_context.cc
+++ b/src/kudu/security/tls_context.cc
@@ -17,6 +17,7 @@
 
 #include "kudu/security/tls_context.h"
 
+#include <mutex>
 #include <string>
 #include <vector>
 
@@ -42,6 +43,7 @@
 
 using strings::Substitute;
 using std::string;
+using std::unique_lock;
 
 DEFINE_int32(ipki_server_key_size, 2048,
              "the number of bits for server cert's private key. The server cert "
@@ -87,7 +89,8 @@ template<> struct SslTypeTraits<X509_STORE_CTX> {
 };
 
 TlsContext::TlsContext()
-    : trusted_cert_count_(0),
+    : lock_(RWMutex::Priority::PREFER_READING),
+      trusted_cert_count_(0),
       has_cert_(false) {
   security::InitializeOpenSSL();
 }
@@ -166,7 +169,7 @@ Status TlsContext::Init() {
   return Status::OK();
 }
 
-Status TlsContext::VerifyCertChain(const Cert& cert) {
+Status TlsContext::VerifyCertChainUnlocked(const Cert& cert) {
   X509_STORE* store = SSL_CTX_get_cert_store(ctx_.get());
   auto store_ctx = ssl_make_unique<X509_STORE_CTX>(X509_STORE_CTX_new());
 
@@ -197,15 +200,16 @@ Status TlsContext::VerifyCertChain(const Cert& cert) {
 }
 
 Status TlsContext::UseCertificateAndKey(const Cert& cert, const PrivateKey& key) {
+  // Verify that the cert and key match.
+  RETURN_NOT_OK(cert.CheckKeyMatch(key));
+
+  std::unique_lock<RWMutex> lock(lock_);
+
   // Verify that the appropriate CA certs have been loaded into the context
   // before we adopt a cert. Otherwise, client connections without the CA cert
   // available would fail.
-  RETURN_NOT_OK(VerifyCertChain(cert));
+  RETURN_NOT_OK(VerifyCertChainUnlocked(cert));
 
-  // Verify that the cert and key match.
-  RETURN_NOT_OK(cert.CheckKeyMatch(key));
-
-  MutexLock lock(lock_);
   CHECK(!has_cert_);
 
   OPENSSL_RET_NOT_OK(SSL_CTX_use_PrivateKey(ctx_.get(), key.GetRawData()),
@@ -219,6 +223,7 @@ Status TlsContext::UseCertificateAndKey(const Cert& cert, const PrivateKey& key)
 Status TlsContext::AddTrustedCertificate(const Cert& cert) {
   VLOG(2) << "Trusting certificate " << cert.SubjectName();
 
+  unique_lock<RWMutex> lock(lock_);
   ERR_clear_error();
   auto* cert_store = SSL_CTX_get_cert_store(ctx_.get());
   int rc = X509_STORE_add_cert(cert_store, cert.GetRawData());
@@ -233,12 +238,13 @@ Status TlsContext::AddTrustedCertificate(const Cert& cert) {
     }
     OPENSSL_RET_NOT_OK(rc, "failed to add trusted certificate");
   }
-  MutexLock lock(lock_);
   trusted_cert_count_ += 1;
   return Status::OK();
 }
 
 Status TlsContext::DumpTrustedCerts(vector<string>* cert_ders) const {
+  shared_lock<RWMutex> lock(lock_);
+
   vector<string> ret;
   auto* cert_store = SSL_CTX_get_cert_store(ctx_.get());
 
@@ -316,7 +322,7 @@ Status TlsContext::GenerateSelfSignedCertAndKey() {
   ignore_result(X509_check_ca(cert.GetRawData()));
 
   // Step 4: Adopt the new key and cert.
-  MutexLock lock(lock_);
+  unique_lock<RWMutex> lock(lock_);
   CHECK(!has_cert_);
   OPENSSL_RET_NOT_OK(SSL_CTX_use_PrivateKey(ctx_.get(), key.GetRawData()),
                      "failed to use private key");
@@ -328,7 +334,7 @@ Status TlsContext::GenerateSelfSignedCertAndKey() {
 }
 
 boost::optional<CertSignRequest> TlsContext::GetCsrIfNecessary() const {
-  MutexLock lock(lock_);
+  shared_lock<RWMutex> lock(lock_);
   if (csr_) {
     return csr_->Clone();
   }
@@ -336,12 +342,12 @@ boost::optional<CertSignRequest> TlsContext::GetCsrIfNecessary() const {
 }
 
 Status TlsContext::AdoptSignedCert(const Cert& cert) {
+  unique_lock<RWMutex> lock(lock_);
+
   // Verify that the appropriate CA certs have been loaded into the context
   // before we adopt a cert. Otherwise, client connections without the CA cert
   // available would fail.
-  RETURN_NOT_OK(VerifyCertChain(cert));
-
-  MutexLock lock(lock_);
+  RETURN_NOT_OK(VerifyCertChainUnlocked(cert));
 
   if (!csr_) {
     // A signed cert has already been adopted.
@@ -392,7 +398,10 @@ Status TlsContext::InitiateHandshake(TlsHandshakeType handshake_type,
                                      TlsHandshake* handshake) const {
   CHECK(ctx_);
   CHECK(!handshake->ssl_);
-  handshake->adopt_ssl(ssl_make_unique(SSL_new(ctx_.get())));
+  {
+    shared_lock<RWMutex> lock(lock_);
+    handshake->adopt_ssl(ssl_make_unique(SSL_new(ctx_.get())));
+  }
   if (!handshake->ssl_) {
     return Status::RuntimeError("failed to create SSL handle", GetOpenSSLErrors());
   }

http://git-wip-us.apache.org/repos/asf/kudu/blob/f726f6ab/src/kudu/security/tls_context.h
----------------------------------------------------------------------
diff --git a/src/kudu/security/tls_context.h b/src/kudu/security/tls_context.h
index 6806b0b..09f35a6 100644
--- a/src/kudu/security/tls_context.h
+++ b/src/kudu/security/tls_context.h
@@ -26,7 +26,8 @@
 #include "kudu/security/cert.h"
 #include "kudu/security/tls_handshake.h"
 #include "kudu/util/atomic.h"
-#include "kudu/util/mutex.h"
+#include "kudu/util/locks.h"
+#include "kudu/util/rw_mutex.h"
 #include "kudu/util/status.h"
 
 namespace kudu {
@@ -75,7 +76,7 @@ class TlsContext {
   // Returns true if this TlsContext has been configured with a cert and key for
   // use with TLS-encrypted connections.
   bool has_cert() const {
-    MutexLock lock(lock_);
+    shared_lock<RWMutex> lock(lock_);
     return has_cert_;
   }
 
@@ -83,13 +84,13 @@ class TlsContext {
   // cert and key for use with TLS-encrypted connections. If this method returns
   // true, then 'has_trusted_cert' will also return true.
   bool has_signed_cert() const {
-    MutexLock lock(lock_);
+    shared_lock<RWMutex> lock(lock_);
     return has_cert_ && !csr_;
   }
 
   // Returns true if this TlsContext has at least one certificate in its trust store.
   bool has_trusted_cert() const {
-    MutexLock lock(lock_);
+    shared_lock<RWMutex> lock(lock_);
     return trusted_cert_count_ > 0;
   }
 
@@ -154,20 +155,21 @@ class TlsContext {
   // Return the number of certs that have been marked as trusted.
   // Used by tests.
   int trusted_cert_count_for_tests() const {
-    MutexLock lock(lock_);
+    shared_lock<RWMutex> lock(lock_);
     return trusted_cert_count_;
   }
 
  private:
 
-  Status VerifyCertChain(const Cert& cert) WARN_UNUSED_RESULT;
+  Status VerifyCertChainUnlocked(const Cert& cert) WARN_UNUSED_RESULT;
 
-  // Owned SSL context.
+  // Protects all members.
+  //
+  // Taken in write mode when any changes are modifying the underlying SSL_CTX
+  // using a mutating method (eg SSL_CTX_use_*) or when changing the value of
+  // any of our own member variables.
+  mutable RWMutex lock_;
   c_unique_ptr<SSL_CTX> ctx_;
-
-  // Protexts trusted_cert_count_, has_cert_ and csr_, as well as ctx_ when it
-  // needs to be updated transactionally with has_cert_ and csr_.
-  mutable Mutex lock_;
   int32_t trusted_cert_count_;
   bool has_cert_;
   boost::optional<CertSignRequest> csr_;

http://git-wip-us.apache.org/repos/asf/kudu/blob/f726f6ab/src/kudu/security/tls_handshake-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/security/tls_handshake-test.cc b/src/kudu/security/tls_handshake-test.cc
index 681e270..60b1b91 100644
--- a/src/kudu/security/tls_handshake-test.cc
+++ b/src/kudu/security/tls_handshake-test.cc
@@ -17,9 +17,12 @@
 
 #include "kudu/security/tls_handshake.h"
 
+#include <atomic>
 #include <functional>
 #include <iostream>
 #include <string>
+#include <thread>
+#include <vector>
 
 #include <boost/optional.hpp>
 #include <gflags/gflags.h>
@@ -29,9 +32,11 @@
 #include "kudu/security/crypto.h"
 #include "kudu/security/security-test-util.h"
 #include "kudu/security/tls_context.h"
+#include "kudu/util/scoped_cleanup.h"
 #include "kudu/util/test_util.h"
 
 using std::string;
+using std::vector;
 
 DECLARE_int32(ipki_server_key_size);
 
@@ -67,8 +72,7 @@ std::ostream& operator<<(std::ostream& o, Case c) {
   return o;
 }
 
-class TestTlsHandshake : public KuduTest,
-                         public ::testing::WithParamInterface<Case> {
+class TestTlsHandshakeBase : public KuduTest {
  public:
   void SetUp() override {
     KuduTest::SetUp();
@@ -125,6 +129,49 @@ class TestTlsHandshake : public KuduTest,
   string key_path_;
 };
 
+class TestTlsHandshake : public TestTlsHandshakeBase,
+                   public ::testing::WithParamInterface<Case> {};
+
+class TestTlsHandshakeConcurrent : public TestTlsHandshakeBase,
+                   public ::testing::WithParamInterface<int> {};
+
+// Test concurrently running handshakes while changing the certificates on the TLS
+// context. We parameterize across different numbers of threads, because surprisingly,
+// fewer threads seems to trigger issues more easily in some cases.
+INSTANTIATE_TEST_CASE_P(NumThreads, TestTlsHandshakeConcurrent, ::testing::Values(1, 2, 4, 8));
+TEST_P(TestTlsHandshakeConcurrent, TestConcurrentAdoptCert) {
+  const int kNumThreads = GetParam();
+
+  ASSERT_OK(server_tls_.GenerateSelfSignedCertAndKey());
+  std::atomic<bool> done(false);
+  vector<std::thread> handshake_threads;
+  for (int i = 0; i < kNumThreads; i++) {
+    handshake_threads.emplace_back([&]() {
+        while (!done) {
+          RunHandshake(TlsVerificationMode::VERIFY_NONE, TlsVerificationMode::VERIFY_NONE);
+        }
+      });
+  }
+  auto c = MakeScopedCleanup([&](){
+      done = true;
+      for (std::thread& t : handshake_threads) {
+        t.join();
+      }
+    });
+
+  SleepFor(MonoDelta::FromMilliseconds(10));
+  {
+    PrivateKey ca_key;
+    Cert ca_cert;
+    ASSERT_OK(GenerateSelfSignedCAForTests(&ca_key, &ca_cert));
+    Cert cert;
+    ASSERT_OK(CertSigner(&ca_cert, &ca_key).Sign(*server_tls_.GetCsrIfNecessary(), &cert));
+    ASSERT_OK(server_tls_.AddTrustedCertificate(ca_cert));
+    ASSERT_OK(server_tls_.AdoptSignedCert(cert));
+  }
+  SleepFor(MonoDelta::FromMilliseconds(10));
+}
+
 TEST_F(TestTlsHandshake, TestHandshakeSequence) {
   PrivateKey ca_key;
   Cert ca_cert;


[2/2] kudu git commit: Workaround a leak in OpenSSL 1.0.0

Posted by da...@apache.org.
Workaround a leak in OpenSSL 1.0.0

This fixes another leak seen occasionally as a flaky test on RHEL 6:

Direct leak of 56 byte(s) in 1 object(s) allocated from:
    #2 0x7f21e5fe037c in CRYPTO_malloc src/openssl/crypto/mem.c:306
    #3 0x7f21e607f45c in EVP_PKEY_new src/openssl/crypto/evp/p_lib.c:186
    #4 0x7f21e608ff33 in X509_PUBKEY_get src/openssl/crypto/asn1/x_pubkey.c:147
    #5 0x7f21e60b23ad in X509_get_pubkey src/openssl/crypto/x509/x509_cmp.c:292
    #6 0x7f21e60b623d in internal_verify src/openssl/crypto/x509/x509_vfy.c:1624
    #7 0x7f21e60b3b8b in X509_verify_cert src/openssl/crypto/x509/x509_vfy.c:372
    #8 0x7f21e5d5c096 in ssl_verify_cert_chain src/openssl/ssl/ssl_cert.c:535
    #9 0x7f21e5d34157 in ssl3_get_server_certificate src/openssl/ssl/s3_clnt.c:1047
    #10 0x7f21e5d3289f in ssl3_connect src/openssl/ssl/s3_clnt.c:310
    #11 0x7f21e5d5705b in SSL_connect src/openssl/ssl/ssl_lib.c:933
    #12 0x7f21e5d43340 in ssl23_get_server_hello src/openssl/ssl/s23_clnt.c:693
    #13 0x7f21e5d422b0 in ssl23_connect src/openssl/ssl/s23_clnt.c:222
    #14 0x7f21e5d59f46 in SSL_do_handshake src/openssl/ssl/ssl_lib.c:2380
    #15 0x7f21e6a028a7 in kudu::security::TlsHandshake::Continue(std::string const&, std::string*) src/kudu/security/tls_handshake.cc:92:12

The issue turned out to be an OpenSSL 1.0.0 bug that was since fixed. But,
since we build/test against OpenSSL 1.0.0 on el6, we hit the leak.

This patch has a workaround - see the included comment for details.

Before the patch, all_types-itest failed about 10% of the time in dist-test,
and now passes reliably.

Change-Id: I65fe981c523f6fe73b3668bfb2f30a95ebf3e759
Reviewed-on: http://gerrit.cloudera.org:8080/6197
Reviewed-by: Dan Burkert <da...@apache.org>
Tested-by: Kudu Jenkins


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

Branch: refs/heads/master
Commit: 0609277c4209c02fd94d7c6fd6c23abd8b1d3b67
Parents: f726f6a
Author: Todd Lipcon <to...@apache.org>
Authored: Tue Feb 28 18:53:35 2017 -0800
Committer: Todd Lipcon <to...@apache.org>
Committed: Wed Mar 1 18:57:13 2017 +0000

----------------------------------------------------------------------
 src/kudu/security/tls_context.cc | 16 ++++++++++++++++
 1 file changed, 16 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/0609277c/src/kudu/security/tls_context.cc
----------------------------------------------------------------------
diff --git a/src/kudu/security/tls_context.cc b/src/kudu/security/tls_context.cc
index 28c8e0e..17b7f69 100644
--- a/src/kudu/security/tls_context.cc
+++ b/src/kudu/security/tls_context.cc
@@ -223,6 +223,22 @@ Status TlsContext::UseCertificateAndKey(const Cert& cert, const PrivateKey& key)
 Status TlsContext::AddTrustedCertificate(const Cert& cert) {
   VLOG(2) << "Trusting certificate " << cert.SubjectName();
 
+  {
+    // Workaround for a leak in OpenSSL <1.0.1:
+    //
+    // If we start trusting a cert, and its internal public-key field hasn't
+    // yet been populated, then the first time it's used for verification will
+    // populate it. In the case that two threads try to populate it at the same time,
+    // one of the thread's copies will be leaked.
+    //
+    // To avoid triggering the race, we populate the internal public key cache
+    // field up front before adding it to the trust store.
+    //
+    // See OpenSSL commit 33a688e80674aaecfac6d9484ec199daa0ee5b61.
+    PublicKey k;
+    CHECK_OK(cert.GetPublicKey(&k));
+  }
+
   unique_lock<RWMutex> lock(lock_);
   ERR_clear_error();
   auto* cert_store = SSL_CTX_get_cert_store(ctx_.get());