You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by ta...@apache.org on 2017/09/05 22:26:53 UTC

[8/9] incubator-impala git commit: IMPALA-5849: Remove compile-time checks for OpenSSL > 1.0.0

IMPALA-5849: Remove compile-time checks for OpenSSL > 1.0.0

(Prerequisite knowledge: Impala must support building and linking
against OpenSSL 1.0.0 and 1.0.1. 1.0.0 only supports TLS1.0. 1.0.1
supports TLS1.1 and 1.2 as well).

Rather than check for OpenSSL w/TLSv1.1+ support at compile time, we
altered Thrift-0.9.0-p11 to compile on all versions, and fail with an
error if an unsupported version is requested at runtime. This patch
is the Impala-side counterpart to that work, and also works around a bug.

The bug is in OpenSSL. When disabling, say, TLS1.1 we pass an option
bitmask to SSL_CTX_set_options() that includes SSL_OP_NO_TLSv1_1. In
OpenSSL 1.0.1 this is defined as 0x10000000L but in 1.0.0 it is not
defined. So to support compilation on all OpenSSL versions, we added
definitions of that constant to Thrift if it was not found at compile
time.

However, in OpenSSL 1.0.0 *another* option for SSL_CTX_set_options() has
the *same* value (0x10000000L). This means that passing
SSL_OP_NO_TLSv1_1 to OpenSSL 1.0.0 actually does something completely
different. This lack of backwards compatibility is the bug.

To work around it, we use the SSLeay() function, which, although
cryptically named, returns the value of OPENSSL_VERSION_NUMBER in the
version of OpenSSL currently linked against (I have tested that it
behaves differently when dynamically linked against different versions
of OpenSSL, but compiled against the same one). We use that method to
tell us which of the TLS protocol versions are actually supported, and
raise errors if they are not.

An alternative approach to doing this inside ThriftClient and
ThriftServer would be to do so in Thrift; this might be a reasonable
future option but for now it is too unwieldy to test a toolchain build
that is linked against different OpenSSL versions. To reduce risk, and
the number of ways things can go wrong, we do all the protocol version
whitelisting in the Impala code.

To simplify the code further, we also remove the values of
SSLProtoVersions that allowed us to restrict the TLS protocol to
exactly *one* value, rather than specify the minimum (e.g. we remove
TLSv1_1 but retain TLSv1_1_plus). The more restrictive options were not
intended for production use, and using them now will raise an error.

Error messages
--------------

Passing a valid, but unsupported version to --ssl_minimum_version:

  "TLS (6) version not supported (linked OpenSSL version is 1234567)"

Passing an invalid version to --ssl_minimum_version:

  "Unknown TLS version: 'tls_4_2'"

Testing
-------

We assume that tests will be run on the same machine that built Impala,
so still execute slightly different tests depending on the detected
OpenSSL version.

For thrift-server-test, we check to see if a protocol version is
supported at runtime, and use that to determine the expected behaviour
of a test which starts all combinations of clients and server versions.

For the webserver-test, we determine at compile-time the set of
supported TLS protocols, and change the test's expected behaviour based
on that.

Tests have been run when compiling and running against OpenSSL 1.0.0 and
OpenSSL 1.0.1.

Also include corresponding changes to Squeasel from this commit:

https://github.com/henryr/squeasel/commit/eded53

Testing: New webserver and thrift-server tests.

Change-Id: I20c5d39c0c4ae9c5445dd3ee3b175fe337a5728d
Reviewed-on: http://gerrit.cloudera.org:8080/7866
Tested-by: Impala Public Jenkins
Reviewed-by: Henry Robinson <he...@cloudera.com>


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

Branch: refs/heads/master
Commit: ac8a72f3041688316a20c9a85eda5a5cb9af1578
Parents: e4a2d22
Author: Henry Robinson <he...@cloudera.com>
Authored: Fri Aug 25 11:39:20 2017 -0700
Committer: Henry Robinson <he...@cloudera.com>
Committed: Sat Sep 2 18:39:21 2017 +0000

----------------------------------------------------------------------
 be/src/rpc/thrift-client.cc           |  6 ++++++
 be/src/rpc/thrift-server-test.cc      | 32 +++++++++++++-----------------
 be/src/rpc/thrift-server.cc           | 21 +++++++++++++++-----
 be/src/rpc/thrift-server.h            |  6 +++++-
 be/src/service/impala-server.cc       |  6 +-----
 be/src/thirdparty/squeasel/squeasel.c | 28 ++++++++++++++++++++++----
 be/src/util/webserver-test.cc         |  9 +++++++++
 bin/impala-config.sh                  |  4 ++--
 8 files changed, 77 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/ac8a72f3/be/src/rpc/thrift-client.cc
----------------------------------------------------------------------
diff --git a/be/src/rpc/thrift-client.cc b/be/src/rpc/thrift-client.cc
index f8da136..d4d246b 100644
--- a/be/src/rpc/thrift-client.cc
+++ b/be/src/rpc/thrift-client.cc
@@ -44,6 +44,12 @@ ThriftClientImpl::ThriftClientImpl(const std::string& ipaddress, int port, bool
     SSLProtocol version;
     init_status_ =
         SSLProtoVersions::StringToProtocol(FLAGS_ssl_minimum_version, &version);
+    if (init_status_.ok() && !SSLProtoVersions::IsSupported(version)) {
+      string err =
+          Substitute("TLS ($0) version not supported (linked OpenSSL version is $1)",
+              version, SSLeay());
+      init_status_ = Status(err);
+    }
     if (!init_status_.ok()) return;
     ssl_factory_.reset(new TSSLSocketFactory(version));
   }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/ac8a72f3/be/src/rpc/thrift-server-test.cc
----------------------------------------------------------------------
diff --git a/be/src/rpc/thrift-server-test.cc b/be/src/rpc/thrift-server-test.cc
index 67e149a..6ceaefd 100644
--- a/be/src/rpc/thrift-server-test.cc
+++ b/be/src/rpc/thrift-server-test.cc
@@ -263,21 +263,12 @@ TEST(SslTest, MismatchedCiphers) {
 // equivalents.
 TEST(SslTest, StringToProtocol) {
   SSLProtocol version;
-#if OPENSSL_VERSION_NUMBER < 0x10001000L
-  // No TLSv1.1+ support in OpenSSL v1.0.0.
-  EXPECT_FALSE(SSLProtoVersions::StringToProtocol("tlsv1.2", &version).ok());
-  EXPECT_FALSE(SSLProtoVersions::StringToProtocol("tlsv1.1", &version).ok());
-  EXPECT_OK(SSLProtoVersions::StringToProtocol("tlsv1", &version));
-  EXPECT_EQ(TLSv1_0_plus, version);
-#else
-  map<string, SSLProtocol> TEST_CASES = {{"tlsv1", TLSv1_0_plus},
-      {"tlsv1.1", TLSv1_1_plus}, {"tlsv1.2", TLSv1_2_plus}, {"tlsv1_only", TLSv1_0},
-      {"tlsv1.1_only", TLSv1_1}, {"tlsv1.2_only", TLSv1_2}};
+  map<string, SSLProtocol> TEST_CASES = {
+      {"tlsv1", TLSv1_0_plus}, {"tlsv1.1", TLSv1_1_plus}, {"tlsv1.2", TLSv1_2_plus}};
   for (auto p : TEST_CASES) {
     EXPECT_OK(SSLProtoVersions::StringToProtocol(p.first, &version));
     EXPECT_EQ(p.second, version) << "TLS version: " << p.first;
   }
-#endif
 }
 
 TEST(SslTest, TLSVersionControl) {
@@ -293,10 +284,9 @@ TEST(SslTest, TLSVersionControl) {
     set<SSLProtocol> whitelist;
   };
 
-#if OPENSSL_VERSION_NUMBER < 0x10001000L
-  vector<Config> configs = {
-      {TLSv1_0, {TLSv1_0, TLSv1_0_plus}}, {TLSv1_0_plus, {TLSv1_0, TLSv1_0_plus}}};
-#else
+  // Test all configurations supported by Thrift, even if some won't work with the linked
+  // OpenSSL(). We catch those by checking IsSupported() for both the client and ther
+  // server.
   vector<Config> configs = {{TLSv1_0, {TLSv1_0, TLSv1_0_plus}},
       {TLSv1_0_plus,
           {TLSv1_0, TLSv1_1, TLSv1_2, TLSv1_0_plus, TLSv1_1_plus, TLSv1_2_plus}},
@@ -304,7 +294,6 @@ TEST(SslTest, TLSVersionControl) {
       {TLSv1_1_plus, {TLSv1_1, TLSv1_2, TLSv1_0_plus, TLSv1_1_plus, TLSv1_2_plus}},
       {TLSv1_2, {TLSv1_2, TLSv1_0_plus, TLSv1_1_plus, TLSv1_2_plus}},
       {TLSv1_2_plus, {TLSv1_2, TLSv1_0_plus, TLSv1_1_plus, TLSv1_2_plus}}};
-#endif
 
   for (const auto& config : configs) {
     // For each config, start a server with the requested protocol spec, and then try to
@@ -317,17 +306,24 @@ TEST(SslTest, TLSVersionControl) {
                   .ssl(SERVER_CERT, PRIVATE_KEY)
                   .ssl_version(config.server_version)
                   .Build(&server));
-    EXPECT_OK(server->Start());
+    if (!SSLProtoVersions::IsSupported(config.server_version)) {
+      EXPECT_FALSE(server->Start().ok());
+      continue;
+    }
+    ASSERT_OK(server->Start());
 
     for (auto client_version : SSLProtoVersions::PROTO_MAP) {
       auto s = ScopedFlagSetter<string>::Make(
           &FLAGS_ssl_minimum_version, client_version.first);
       ThriftClient<StatestoreServiceClientWrapper> ssl_client(
           "localhost", port, "", nullptr, true);
+      if (!SSLProtoVersions::IsSupported(client_version.second)) {
+        EXPECT_FALSE(ssl_client.Open().ok());
+        continue;
+      }
       EXPECT_OK(ssl_client.Open());
       bool send_done = false;
       TRegisterSubscriberResponse resp;
-
       if (config.whitelist.find(client_version.second) == config.whitelist.end()) {
         EXPECT_THROW(ssl_client.iface()->RegisterSubscriber(
                          resp, TRegisterSubscriberRequest(), &send_done),

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/ac8a72f3/be/src/rpc/thrift-server.cc
----------------------------------------------------------------------
diff --git a/be/src/rpc/thrift-server.cc b/be/src/rpc/thrift-server.cc
index 35a1852..467c004 100644
--- a/be/src/rpc/thrift-server.cc
+++ b/be/src/rpc/thrift-server.cc
@@ -77,11 +77,7 @@ namespace impala {
 // that one (e.g. TLSv1.1 enables v1.1 and v1.2). Specifying TLSv1.1_only enables only
 // v1.1.
 map<string, SSLProtocol> SSLProtoVersions::PROTO_MAP = {
-#if OPENSSL_VERSION_NUMBER >= 0x10001000L
-    {"tlsv1.2", TLSv1_2_plus}, {"tlsv1.2_only", TLSv1_2}, {"tlsv1.1", TLSv1_1_plus},
-    {"tlsv1.1_only", TLSv1_1},
-#endif
-    {"tlsv1", TLSv1_0_plus}, {"tlsv1_only", TLSv1_0}};
+    {"tlsv1.2", TLSv1_2_plus}, {"tlsv1.1", TLSv1_1_plus}, {"tlsv1", TLSv1_0_plus}};
 
 Status SSLProtoVersions::StringToProtocol(const string& in, SSLProtocol* protocol) {
   for (const auto& proto : SSLProtoVersions::PROTO_MAP) {
@@ -94,6 +90,16 @@ Status SSLProtoVersions::StringToProtocol(const string& in, SSLProtocol* protoco
   return Status(Substitute("Unknown TLS version: '$0'", in));
 }
 
+#define OPENSSL_MIN_VERSION_WITH_TLS_1_1 0x10001000L
+
+bool SSLProtoVersions::IsSupported(const SSLProtocol& protocol) {
+  bool is_openssl_1_0_0_or_lower = (SSLeay() < OPENSSL_MIN_VERSION_WITH_TLS_1_1);
+  if (is_openssl_1_0_0_or_lower) return (protocol == TLSv1_0_plus);
+
+  // All other versions supported by OpenSSL 1.0.1 and later.
+  return true;
+}
+
 bool EnableInternalSslConnections() {
   // Enable SSL between servers only if both the client validation certificate and the
   // server certificate are specified. 'Client' here means clients that are used by Impala
@@ -374,6 +380,11 @@ class ImpalaSslSocketFactory : public TSSLSocketFactory {
 }
 Status ThriftServer::CreateSocket(boost::shared_ptr<TServerTransport>* socket) {
   if (ssl_enabled()) {
+    if (!SSLProtoVersions::IsSupported(version_)) {
+      return Status(TErrorCode::SSL_SOCKET_CREATION_FAILED,
+          Substitute("TLS ($0) version not supported (linked OpenSSL version is $1)",
+                        version_, SSLeay()));
+    }
     try {
       // This 'factory' is only called once, since CreateSocket() is only called from
       // Start(). The c'tor may throw if there is an error initializing the SSL context.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/ac8a72f3/be/src/rpc/thrift-server.h
----------------------------------------------------------------------
diff --git a/be/src/rpc/thrift-server.h b/be/src/rpc/thrift-server.h
index e486e37..2002f7f 100644
--- a/be/src/rpc/thrift-server.h
+++ b/be/src/rpc/thrift-server.h
@@ -340,7 +340,7 @@ class ThriftServerBuilder {
 
   bool enable_ssl_ = false;
   apache::thrift::transport::SSLProtocol version_ =
-      apache::thrift::transport::SSLProtocol::TLSv1_0;
+      apache::thrift::transport::SSLProtocol::TLSv1_0_plus;
   std::string certificate_;
   std::string private_key_;
   std::string pem_password_cmd_;
@@ -355,6 +355,10 @@ struct SSLProtoVersions {
   /// one cannot be found. Matching is case-insensitive.
   static Status StringToProtocol(
       const std::string& in, apache::thrift::transport::SSLProtocol* protocol);
+
+  /// Returns true if 'protocol' is supported by the version of OpenSSL this binary is
+  /// linked to.
+  static bool IsSupported(const apache::thrift::transport::SSLProtocol& protocol);
 };
 
 // Returns true if, per the process configuration flags, server<->server communications

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/ac8a72f3/be/src/service/impala-server.cc
----------------------------------------------------------------------
diff --git a/be/src/service/impala-server.cc b/be/src/service/impala-server.cc
index 4d55c85..18582da 100644
--- a/be/src/service/impala-server.cc
+++ b/be/src/service/impala-server.cc
@@ -183,11 +183,7 @@ DEFINE_string(ssl_cipher_list, "",
 
 const string SSL_MIN_VERSION_HELP = "The minimum SSL/TLS version that Thrift "
     "services should use for both client and server connections. Supported versions are "
-#if OPENSSL_VERSION_NUMBER >= 0x10001000L
-    "TLSv1.0, TLSv1.1 and TLSv1.2";
-#else
-    "TLSv1.0";
-#endif
+    "TLSv1.0, TLSv1.1 and TLSv1.2 (as long as the system OpenSSL library supports them)";
 DEFINE_string(ssl_minimum_version, "tlsv1", SSL_MIN_VERSION_HELP.c_str());
 
 DEFINE_int32(idle_session_timeout, 0, "The time, in seconds, that a session may be idle"

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/ac8a72f3/be/src/thirdparty/squeasel/squeasel.c
----------------------------------------------------------------------
diff --git a/be/src/thirdparty/squeasel/squeasel.c b/be/src/thirdparty/squeasel/squeasel.c
index 055a959..d461b24 100644
--- a/be/src/thirdparty/squeasel/squeasel.c
+++ b/be/src/thirdparty/squeasel/squeasel.c
@@ -150,10 +150,20 @@ typedef int socklen_t;
 
 static const char *http_500_error = "Internal Server Error";
 
+#include <openssl/crypto.h>
 #include <openssl/ssl.h>
 #include <openssl/err.h>
 
-#define OPENSSL_VERSION_HAS_TLS_1_1 0x10001000L
+// If these constants aren't defined, we still need them to compile and maintain backwards
+// compatibility with pre-1.0.1 OpenSSL.
+#ifndef SSL_OP_NO_TLSv1
+#define SSL_OP_NO_TLSv1 0x04000000U
+#endif
+#ifndef SSL_OP_NO_TLSv1_1
+#define SSL_OP_NO_TLSv1_1 0x10000000U
+#endif
+
+#define OPENSSL_MIN_VERSION_WITH_TLS_1_1 0x10001000L
 
 static const char *month_names[] = {
   "Jan", "Feb", "Mar", "Apr", "May", "Jun",
@@ -4224,12 +4234,18 @@ static int set_ssl_option(struct sq_context *ctx) {
   int options = SSL_OP_NO_SSLv2 | SSL_OP_NO_SSLv3;
   if (sq_strcasecmp(ssl_version, "tlsv1") == 0) {
     // No-op - don't exclude any TLS protocols.
-#if OPENSSL_VERSION_NUMBER >= OPENSSL_VERSION_HAS_TLS_1_1
   } else if (sq_strcasecmp(ssl_version, "tlsv1.1") == 0) {
+    if (SSLeay() < OPENSSL_MIN_VERSION_WITH_TLS_1_1) {
+      cry(fc(ctx), "Unsupported TLS version: %s", ssl_version);
+      return 0;
+    }
     options |= SSL_OP_NO_TLSv1;
   } else if (sq_strcasecmp(ssl_version, "tlsv1.2") == 0) {
+    if (SSLeay() < OPENSSL_MIN_VERSION_WITH_TLS_1_1) {
+      cry(fc(ctx), "Unsupported TLS version: %s", ssl_version);
+      return 0;
+    }
     options |= (SSL_OP_NO_TLSv1 | SSL_OP_NO_TLSv1_1);
-#endif
   } else {
     cry(fc(ctx), "%s: unknown SSL version: %s", __func__, ssl_version);
     return 0;
@@ -4251,7 +4267,11 @@ static int set_ssl_option(struct sq_context *ctx) {
     return 0;
   }
 
-  SSL_CTX_set_options(ctx->ssl_ctx, options);
+  if ((SSL_CTX_set_options(ctx->ssl_ctx, options) & options) != options) {
+    cry(fc(ctx), "SSL_CTX_set_options (server) error: could not set options (%d)",
+        options);
+    return 0;
+  }
 
   if (ctx->config[SSL_PRIVATE_KEY_PASSWORD] != NULL) {
     SSL_CTX_set_default_passwd_cb(ctx->ssl_ctx, ssl_password_callback);

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/ac8a72f3/be/src/util/webserver-test.cc
----------------------------------------------------------------------
diff --git a/be/src/util/webserver-test.cc b/be/src/util/webserver-test.cc
index ba6f276..78934c6 100644
--- a/be/src/util/webserver-test.cc
+++ b/be/src/util/webserver-test.cc
@@ -297,8 +297,10 @@ TEST(Webserver, SslGoodTlsVersion) {
 
 #if OPENSSL_VERSION_NUMBER >= 0x10001000L
   auto versions = {"tlsv1", "tlsv1.1", "tlsv1.2"};
+  vector<string> unsupported_versions = {};
 #else
   auto versions = {"tlsv1"};
+  auto unsupported_versions = {"tlsv1.1", "tlsv1.2"};
 #endif
   for (auto v: versions) {
     auto ssl_version = ScopedFlagSetter<string>::Make(
@@ -307,6 +309,13 @@ TEST(Webserver, SslGoodTlsVersion) {
     Webserver webserver(FLAGS_webserver_port);
     ASSERT_OK(webserver.Start());
   }
+
+  for (auto v : unsupported_versions) {
+    auto ssl_version = ScopedFlagSetter<string>::Make(&FLAGS_ssl_minimum_version, v);
+
+    Webserver webserver(FLAGS_webserver_port);
+    EXPECT_FALSE(webserver.Start().ok()) << "Version: " << v;
+  }
 }
 
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/ac8a72f3/bin/impala-config.sh
----------------------------------------------------------------------
diff --git a/bin/impala-config.sh b/bin/impala-config.sh
index 15951fd..f3354f1 100755
--- a/bin/impala-config.sh
+++ b/bin/impala-config.sh
@@ -72,7 +72,7 @@ fi
 # moving to a different build of the toolchain, e.g. when a version is bumped or a
 # compile option is changed. The build id can be found in the output of the toolchain
 # build jobs, it is constructed from the build number and toolchain git hash prefix.
-export IMPALA_TOOLCHAIN_BUILD_ID=457-22902e9fb8
+export IMPALA_TOOLCHAIN_BUILD_ID=459-0157f69796
 # Versions of toolchain dependencies.
 # -----------------------------------
 export IMPALA_AVRO_VERSION=1.7.4-p4
@@ -107,7 +107,7 @@ export IMPALA_SQUEASEL_VERSION=3.3
 # TPC utilities used for test/benchmark data generation.
 export IMPALA_TPC_DS_VERSION=2.1.0
 export IMPALA_TPC_H_VERSION=2.17.0
-export IMPALA_THRIFT_VERSION=0.9.0-p10
+export IMPALA_THRIFT_VERSION=0.9.0-p11
 export IMPALA_THRIFT_JAVA_VERSION=0.9.0
 export IMPALA_ZLIB_VERSION=1.2.8