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

kudu git commit: KUDU-1738. Allow users of the client to disable OpenSSL initialization

Repository: kudu
Updated Branches:
  refs/heads/master 93ec241a9 -> b73a714b9


KUDU-1738. Allow users of the client to disable OpenSSL initialization

OpenSSL's initialization sequence is not thread-safe, and many
applications that embed Kudu may also be initializing OpenSSL by some
other means. This provides an API so that such applications can disable
Kudu's initialization sequence.

This patch exposed a couple cases where we previously had conflicting
OpenSSL initializations:

* EasyCurl was initializing curl, which initted OpenSSL.

The fix for this one was to explicitly disable curl's OpenSSL init
sequence.

* Python tests failed because _ssl was getting imported by setuptools.

The fix for this one was to explicitly disable Kudu's init sequence.

Additionally, Matt Jacobs noticed that the THREADID callback is actually
not necessary on Linux -- if it isn't set, OpenSSL since 1.0.0 (our min
version) defaults to using &errno, which is a different address in each
thread. So, this patch removes the setting of this callback.

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


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

Branch: refs/heads/master
Commit: b73a714b97a32b2a9cd90e7e89c5b1be65eb31c1
Parents: 93ec241
Author: Todd Lipcon <to...@apache.org>
Authored: Mon Feb 13 18:17:24 2017 -0800
Committer: Todd Lipcon <to...@apache.org>
Committed: Fri Feb 17 04:58:04 2017 +0000

----------------------------------------------------------------------
 CMakeLists.txt                            |  2 +-
 python/kudu/client.pyx                    | 10 ++++
 python/kudu/libkudu_client.pxd            |  2 +
 src/kudu/client/client-unittest.cc        |  7 +++
 src/kudu/client/client.cc                 |  5 ++
 src/kudu/client/client.h                  | 25 ++++++++
 src/kudu/integration-tests/CMakeLists.txt |  1 +
 src/kudu/master/CMakeLists.txt            |  8 ++-
 src/kudu/security/openssl_util.cc         | 80 ++++++++++++++++++++++----
 src/kudu/security/openssl_util.h          |  9 +++
 src/kudu/security/tls_context.cc          |  3 -
 src/kudu/server/CMakeLists.txt            |  5 +-
 src/kudu/tserver/CMakeLists.txt           |  1 +
 src/kudu/util/CMakeLists.txt              | 15 ++++-
 src/kudu/util/curl_util.cc                |  8 ++-
 15 files changed, 159 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/b73a714b/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/CMakeLists.txt b/CMakeLists.txt
index 3ee9666..72cf219 100644
--- a/CMakeLists.txt
+++ b/CMakeLists.txt
@@ -844,8 +844,8 @@ ADD_THIRDPARTY_LIB(squeasel
 ##
 ## Version 1.0.0 or higher is required because we are using the following
 ## features introduced started OpenSSL 1.0.0:
-##   * CRYPTO_THREADID and associated functions
 ##   * The new breed of functions to work with the X509_EXTENSION stack
+##   * automatic usage of &errno as a safe per-thread identifier
 ##
 ## If having multiple OpenSSL libraries installed on the system,
 ## use the OPENSSL_ROOT_DIR cmake flag to specify where to look for the proper

http://git-wip-us.apache.org/repos/asf/kudu/blob/b73a714b/python/kudu/client.pyx
----------------------------------------------------------------------
diff --git a/python/kudu/client.pyx b/python/kudu/client.pyx
index c95844d..0b28856 100644
--- a/python/kudu/client.pyx
+++ b/python/kudu/client.pyx
@@ -244,6 +244,16 @@ cdef class Client:
             KuduClientBuilder builder
             TimeDelta timeout
 
+        # Python programs will often have already imported _ssl, which
+        # has the side effect of initializing OpenSSL. So, we detect
+        # whether _ssl is present, and if we can import it, we disable
+        # Kudu's initialization to avoid a conflict.
+        try:
+          import _ssl
+          DisableOpenSSLInitialization()
+        except:
+          pass
+
         if isinstance(addr_or_addrs, six.string_types):
             addr_or_addrs = [addr_or_addrs]
         elif not isinstance(addr_or_addrs, list):

http://git-wip-us.apache.org/repos/asf/kudu/blob/b73a714b/python/kudu/libkudu_client.pxd
----------------------------------------------------------------------
diff --git a/python/kudu/libkudu_client.pxd b/python/kudu/libkudu_client.pxd
index 424380f..9d1dfb7 100644
--- a/python/kudu/libkudu_client.pxd
+++ b/python/kudu/libkudu_client.pxd
@@ -469,6 +469,8 @@ cdef extern from "kudu/client/client.h" namespace "kudu::client" nogil:
         PartitionType_Exclusive " kudu::client::KuduTableCreator::EXCLUSIVE_BOUND"
         PartitionType_Inclusive " kudu::client::KuduTableCreator::INCLUSIVE_BOUND"
 
+    Status DisableOpenSSLInitialization()
+
     cdef cppclass KuduClient:
 
         Status DeleteTable(const string& table_name)

http://git-wip-us.apache.org/repos/asf/kudu/blob/b73a714b/src/kudu/client/client-unittest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/client/client-unittest.cc b/src/kudu/client/client-unittest.cc
index 403044c..ae2c623 100644
--- a/src/kudu/client/client-unittest.cc
+++ b/src/kudu/client/client-unittest.cc
@@ -156,6 +156,13 @@ TEST(ClientUnitTest, TestSchemaBuilder_CompoundKey_BadColumnName) {
             b.Build(&s).ToString());
 }
 
+TEST(ClientUnitTest, TestDisableSslFailsIfNotInitialized) {
+  // If we try to disable SSL initialization without setting up SSL properly,
+  // it should return an error.
+  Status s = DisableOpenSSLInitialization();
+  ASSERT_STR_MATCHES(s.ToString(), "Locking callback not initialized");
+}
+
 namespace {
 Status TestFunc(const MonoTime& deadline, bool* retry, int* counter) {
   (*counter)++;

http://git-wip-us.apache.org/repos/asf/kudu/blob/b73a714b/src/kudu/client/client.cc
----------------------------------------------------------------------
diff --git a/src/kudu/client/client.cc b/src/kudu/client/client.cc
index 99b37f2..720d6ef 100644
--- a/src/kudu/client/client.cc
+++ b/src/kudu/client/client.cc
@@ -60,6 +60,7 @@
 #include "kudu/rpc/messenger.h"
 #include "kudu/rpc/request_tracker.h"
 #include "kudu/rpc/sasl_common.h"
+#include "kudu/security/openssl_util.h"
 #include "kudu/util/init.h"
 #include "kudu/util/logging.h"
 #include "kudu/util/net/dns_resolver.h"
@@ -185,6 +186,10 @@ Status DisableSaslInitialization() {
   return kudu::rpc::DisableSaslInitialization();
 }
 
+Status DisableOpenSSLInitialization() {
+  return kudu::security::DisableOpenSSLInitialization();
+}
+
 string GetShortVersionString() {
   return VersionInfo::GetShortVersionString();
 }

http://git-wip-us.apache.org/repos/asf/kudu/blob/b73a714b/src/kudu/client/client.h
----------------------------------------------------------------------
diff --git a/src/kudu/client/client.h b/src/kudu/client/client.h
index 2f51d3d..f8e0c73 100644
--- a/src/kudu/client/client.h
+++ b/src/kudu/client/client.h
@@ -141,6 +141,31 @@ Status KUDU_EXPORT SetInternalSignalNumber(int signum);
 /// implementation if they are choosing to handle SASL initialization manually.
 Status KUDU_EXPORT DisableSaslInitialization();
 
+
+/// Disable initialization of the OpenSSL library. Clients should call this
+/// method and manually initialize OpenSSL before using the Kudu client if
+/// they are also using OpenSSL for any other purpose. If this method is not
+/// called, Kudu will attempt to initialize OpenSSL, which may trigger a crash
+/// if concurrent with another thread's initialization attempt.
+///
+/// If this function is called, it must be called prior to the first construction
+/// of a KuduClient object.
+///
+/// @note If OpenSSL initialization is disabled, Kudu depends on the embedding
+/// application to take care of initialization. When this function is called,
+/// Kudu will attempt to verify that the appropriate initialization steps have
+/// been taken, and return a bad Status if they have not. Applications may
+/// use the following code to initialize OpenSSL:
+///
+/// @code
+///   SSL_load_error_strings();
+///   SSL_library_init();
+///   OpenSSL_add_all_algorithms();
+///   RAND_poll(); // or an equivalent RAND setup.
+///   CRYPTO_set_locking_callback(MyAppLockingCallback);
+/// @endcode
+Status KUDU_EXPORT DisableOpenSSLInitialization();
+
 /// @return Short version info, i.e. a single-line version string
 ///   identifying the Kudu client.
 std::string KUDU_EXPORT GetShortVersionString();

http://git-wip-us.apache.org/repos/asf/kudu/blob/b73a714b/src/kudu/integration-tests/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/CMakeLists.txt b/src/kudu/integration-tests/CMakeLists.txt
index f3e9019..8c6da5e 100644
--- a/src/kudu/integration-tests/CMakeLists.txt
+++ b/src/kudu/integration-tests/CMakeLists.txt
@@ -39,6 +39,7 @@ target_link_libraries(integration-tests
   ksck
   kudu_client
   kudu_client_test_util
+  kudu_curl_util
   kudu_fs
   kudu_test_util
   security-test)

http://git-wip-us.apache.org/repos/asf/kudu/blob/b73a714b/src/kudu/master/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/kudu/master/CMakeLists.txt b/src/kudu/master/CMakeLists.txt
index 673132e..54ee218 100644
--- a/src/kudu/master/CMakeLists.txt
+++ b/src/kudu/master/CMakeLists.txt
@@ -63,7 +63,13 @@ target_link_libraries(master
   tserver_service_proto)
 
 # Tests
-set(KUDU_TEST_LINK_LIBS master master_proto kudu_client ${KUDU_MIN_TEST_LIBS})
+set(KUDU_TEST_LINK_LIBS
+  ${KUDU_MIN_TEST_LIBS}
+  kudu_client
+  kudu_curl_util
+  master
+  master_proto)
+
 ADD_KUDU_TEST(catalog_manager-test)
 ADD_KUDU_TEST(master-test RESOURCE_LOCK "master-web-port")
 ADD_KUDU_TEST(sys_catalog-test RESOURCE_LOCK "master-web-port")

http://git-wip-us.apache.org/repos/asf/kudu/blob/b73a714b/src/kudu/security/openssl_util.cc
----------------------------------------------------------------------
diff --git a/src/kudu/security/openssl_util.cc b/src/kudu/security/openssl_util.cc
index ea4a259..cdff7e9 100644
--- a/src/kudu/security/openssl_util.cc
+++ b/src/kudu/security/openssl_util.cc
@@ -42,6 +42,20 @@ namespace security {
 
 namespace {
 
+// Determine whether initialization was ever called.
+//
+// Thread safety:
+// - written by DoInitializeOpenSSL (single-threaded, due to std::call_once)
+// - read by DisableOpenSSLInitialization (must not be concurrent with above)
+bool g_ssl_is_initialized = false;
+
+// If true, then we expect someone else has initialized SSL.
+//
+// Thread safety:
+// - read by DoInitializeOpenSSL (single-threaded, due to std::call_once)
+// - written by DisableOpenSSLInitialization (must not be concurrent with above)
+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.
 Mutex* kCryptoLocks = nullptr;
@@ -57,31 +71,73 @@ void LockingCB(int mode, int type, const char* /*file*/, int /*line*/) {
   }
 }
 
-// Return the current pthread's tid. Only to be used by OpenSSL.
-void ThreadIdCB(CRYPTO_THREADID* tid) {
-  CRYPTO_THREADID_set_numeric(tid, Thread::UniqueThreadId());
+Status CheckOpenSSLInitialized() {
+  if (!CRYPTO_get_locking_callback()) {
+    return Status::RuntimeError("Locking callback not initialized");
+  }
+  auto ctx = ssl_make_unique(SSL_CTX_new(SSLv23_method()));
+  if (!ctx) {
+    return Status::RuntimeError("SSL library appears uninitialized (cannot create SSL_CTX)");
+  }
+  return Status::OK();
 }
 
 void DoInitializeOpenSSL() {
+  if (g_disable_ssl_init) {
+    VLOG(2) << "Not initializing OpenSSL (disabled by application)";
+    return;
+  }
+
+  // Check that OpenSSL isn't already initialized. If it is, it's likely
+  // we are embedded in (or embedding) another application/library which
+  // initializes OpenSSL, and we risk installing conflicting callbacks
+  // or crashing due to concurrent initialization attempts. In that case,
+  // log a warning.
+  auto ctx = ssl_make_unique(SSL_CTX_new(SSLv23_method()));
+  if (ctx) {
+    LOG(DFATAL) << "It appears that OpenSSL has been previously initialized by "
+                << "code outside of Kudu. Please use kudu::client::DisableOpenSSLInitialization() "
+                << "to avoid potential crashes due to conflicting initialization.";
+    // Continue anyway rather than crashing the process in release builds.
+    // All of the below is idempotent, except for the locking callback, which we
+    // check before overriding. They aren't thread-safe, however -- that's why
+    // we try to get embedding applications to do the right thing here rather
+    // than risk a potential initialization race.
+  }
+
   SSL_load_error_strings();
   SSL_library_init();
   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();
-  CHECK(!kCryptoLocks);
-  kCryptoLocks = new Mutex[num_locks];
+  if (!CRYPTO_get_locking_callback()) {
+    // Initialize the OpenSSL mutexes. We intentionally leak these, so ignore
+    // LSAN warnings.
+    debug::ScopedLeakCheckDisabler d;
+    int num_locks = CRYPTO_num_locks();
+    CHECK(!kCryptoLocks);
+    kCryptoLocks = new Mutex[num_locks];
+
+    // Callbacks used by OpenSSL required in a multi-threaded setting.
+    CRYPTO_set_locking_callback(LockingCB);
+  }
 
-  // Callbacks used by OpenSSL required in a multi-threaded setting.
-  CRYPTO_set_locking_callback(LockingCB);
-  CRYPTO_THREADID_set_callback(ThreadIdCB);
+  g_ssl_is_initialized = true;
 }
 
 } // anonymous namespace
 
+Status DisableOpenSSLInitialization() {
+  if (g_disable_ssl_init) return Status::OK();
+  if (g_ssl_is_initialized) {
+    return Status::IllegalState("SSL already initialized. Initialization can only be disabled "
+                                "before first usage.");
+  }
+  RETURN_NOT_OK(CheckOpenSSLInitialized());
+  g_disable_ssl_init = true;
+  return Status::OK();
+}
+
 void InitializeOpenSSL() {
   static std::once_flag ssl_once;
   std::call_once(ssl_once, DoInitializeOpenSSL);

http://git-wip-us.apache.org/repos/asf/kudu/blob/b73a714b/src/kudu/security/openssl_util.h
----------------------------------------------------------------------
diff --git a/src/kudu/security/openssl_util.h b/src/kudu/security/openssl_util.h
index c1a4ecf..5830da4 100644
--- a/src/kudu/security/openssl_util.h
+++ b/src/kudu/security/openssl_util.h
@@ -22,6 +22,7 @@
 #include <string>
 
 #include <openssl/pem.h>
+#include <openssl/ssl.h>
 #include <openssl/x509.h>
 
 #include "kudu/gutil/strings/substitute.h"
@@ -51,7 +52,12 @@ typedef struct x509_st X509;
 namespace kudu {
 namespace security {
 
+// Disable initialization of OpenSSL. Must be called before
+// any call to InitializeOpenSSL().
+Status DisableOpenSSLInitialization();
+
 // Initializes static state required by the OpenSSL library.
+// This is a no-op if DisableOpenSSLInitialization() has been called.
 //
 // Safe to call multiple times.
 void InitializeOpenSSL();
@@ -103,6 +109,9 @@ template<> struct SslTypeTraits<X509_REQ> {
 template<> struct SslTypeTraits<EVP_PKEY> {
   static constexpr auto free = &EVP_PKEY_free;
 };
+template<> struct SslTypeTraits<SSL_CTX> {
+  static constexpr auto free = &SSL_CTX_free;
+};
 
 template<typename SSL_TYPE, typename Traits = SslTypeTraits<SSL_TYPE>>
 c_unique_ptr<SSL_TYPE> ssl_make_unique(SSL_TYPE* d) {

http://git-wip-us.apache.org/repos/asf/kudu/blob/b73a714b/src/kudu/security/tls_context.cc
----------------------------------------------------------------------
diff --git a/src/kudu/security/tls_context.cc b/src/kudu/security/tls_context.cc
index cae2b38..f68e581 100644
--- a/src/kudu/security/tls_context.cc
+++ b/src/kudu/security/tls_context.cc
@@ -50,9 +50,6 @@ using ca::CertRequestGenerator;
 template<> struct SslTypeTraits<SSL> {
   static constexpr auto free = &SSL_free;
 };
-template<> struct SslTypeTraits<SSL_CTX> {
-  static constexpr auto free = &SSL_CTX_free;
-};
 template<> struct SslTypeTraits<X509_STORE_CTX> {
   static constexpr auto free = &X509_STORE_CTX_free;
 };

http://git-wip-us.apache.org/repos/asf/kudu/blob/b73a714b/src/kudu/server/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/kudu/server/CMakeLists.txt b/src/kudu/server/CMakeLists.txt
index 25ce3df..4d7bcf2 100644
--- a/src/kudu/server/CMakeLists.txt
+++ b/src/kudu/server/CMakeLists.txt
@@ -102,5 +102,8 @@ endif()
 # server_process tests
 #########################################
 
-set(KUDU_TEST_LINK_LIBS server_process security-test ${KUDU_MIN_TEST_LIBS})
+set(KUDU_TEST_LINK_LIBS  ${KUDU_MIN_TEST_LIBS}
+  kudu_curl_util
+  server_process
+  security-test)
 ADD_KUDU_TEST(webserver-test)

http://git-wip-us.apache.org/repos/asf/kudu/blob/b73a714b/src/kudu/tserver/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/CMakeLists.txt b/src/kudu/tserver/CMakeLists.txt
index 70fa9c8..8cf2ea6 100644
--- a/src/kudu/tserver/CMakeLists.txt
+++ b/src/kudu/tserver/CMakeLists.txt
@@ -162,6 +162,7 @@ target_link_libraries(tserver_test_util
 #########################################
 
 set(KUDU_TEST_LINK_LIBS
+  kudu_curl_util
   tserver
   tserver_test_util
   ${KUDU_MIN_TEST_LIBS})

http://git-wip-us.apache.org/repos/asf/kudu/blob/b73a714b/src/kudu/util/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/kudu/util/CMakeLists.txt b/src/kudu/util/CMakeLists.txt
index 719af21..27f5d70 100644
--- a/src/kudu/util/CMakeLists.txt
+++ b/src/kudu/util/CMakeLists.txt
@@ -266,10 +266,8 @@ ADD_EXPORTABLE_LIBRARY(kudu_util_compression
 
 if(NOT NO_TESTS)
   add_library(kudu_test_util
-    test_util.cc
-    curl_util.cc)
+    test_util.cc)
   target_link_libraries(kudu_test_util
-    ${CURL_LIBRARIES}
     gflags
     glog
     gmock
@@ -281,6 +279,17 @@ if(NOT NO_TESTS)
   endif()
 
 #######################################
+# kudu_curl_util
+#######################################
+  add_library(kudu_curl_util
+    curl_util.cc)
+  target_link_libraries(kudu_curl_util
+    security
+    ${CURL_LIBRARIES}
+    glog
+    gutil)
+
+#######################################
 # kudu_test_main
 #######################################
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/b73a714b/src/kudu/util/curl_util.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/curl_util.cc b/src/kudu/util/curl_util.cc
index c3bfdb6..133e2e6 100644
--- a/src/kudu/util/curl_util.cc
+++ b/src/kudu/util/curl_util.cc
@@ -17,11 +17,13 @@
 
 #include "kudu/util/curl_util.h"
 
-#include "kudu/gutil/strings/substitute.h"
 
 #include <curl/curl.h>
 #include <glog/logging.h>
 
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/security/openssl_util.h"
+
 namespace kudu {
 
 namespace {
@@ -45,6 +47,10 @@ size_t WriteCallback(void* buffer, size_t size, size_t nmemb, void* user_ptr) {
 } // anonymous namespace
 
 EasyCurl::EasyCurl() {
+  // Use our own SSL initialization, and disable curl's.
+  // Both of these calls are idempotent.
+  security::InitializeOpenSSL();
+  CHECK_EQ(0, curl_global_init(CURL_GLOBAL_DEFAULT & ~CURL_GLOBAL_SSL));
   curl_ = curl_easy_init();
   CHECK(curl_) << "Could not init curl";
 }