You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by he...@apache.org on 2017/08/15 17:59:39 UTC

[4/7] incubator-impala git commit: IMPALA-5743: Support TLS version configuration for Thrift servers

IMPALA-5743: Support TLS version configuration for Thrift servers

* Add --ssl_minimum_version which controls the minimum SSL/TLS version
  that clients and servers will use when negotiating a secure
  connection.
* Two kinds of version specification are allowed: 'TLSv1.1' enables
  TLSv1.1 and all subsequent verisons. 'TLSv1.1_only' enables only
  TLSv1.1. The latter is not exposed in user-facing text as it is
  typically only used for testing.
* Handle case where platform may not support TLSv1.1 or v1.2 by checking
  OpenSSL version number.
* Bump Thrift toolchain version to -p10.

Testing:
* New tests in thrift-server-test.cc. In particular, test all 36
  configurations of client and server protocol versions, and ensure that
  the expected successes or failures are seen.

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


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

Branch: refs/heads/master
Commit: 16ce201f5250451cb55e2cb9821b5d628d777160
Parents: 97803ce
Author: Henry Robinson <he...@cloudera.com>
Authored: Tue Aug 1 13:41:34 2017 -0700
Committer: Impala Public Jenkins <im...@gerrit.cloudera.org>
Committed: Tue Aug 15 09:28:05 2017 +0000

----------------------------------------------------------------------
 be/src/catalog/catalogd-main.cc            |  6 ++
 be/src/rpc/thrift-client.cc                | 14 ++++
 be/src/rpc/thrift-client.h                 |  6 +-
 be/src/rpc/thrift-server-test.cc           | 88 +++++++++++++++++++++++++
 be/src/rpc/thrift-server.cc                | 43 ++++++++++--
 be/src/rpc/thrift-server.h                 | 49 ++++++++++----
 be/src/service/impala-server.cc            | 19 ++++++
 be/src/statestore/statestore-subscriber.cc |  6 ++
 be/src/statestore/statestored-main.cc      |  6 ++
 bin/impala-config.sh                       |  4 +-
 10 files changed, 215 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/16ce201f/be/src/catalog/catalogd-main.cc
----------------------------------------------------------------------
diff --git a/be/src/catalog/catalogd-main.cc b/be/src/catalog/catalogd-main.cc
index 4847920..fe681d0 100644
--- a/be/src/catalog/catalogd-main.cc
+++ b/be/src/catalog/catalogd-main.cc
@@ -48,11 +48,13 @@ DECLARE_string(ssl_server_certificate);
 DECLARE_string(ssl_private_key);
 DECLARE_string(ssl_private_key_password_cmd);
 DECLARE_string(ssl_cipher_list);
+DECLARE_string(ssl_minimum_version);
 
 #include "common/names.h"
 
 using namespace impala;
 using namespace apache::thrift;
+using namespace apache::thrift;
 
 int CatalogdMain(int argc, char** argv) {
   FLAGS_webserver_port = 25020;
@@ -94,9 +96,13 @@ int CatalogdMain(int argc, char** argv) {
   ThriftServerBuilder builder("CatalogService", processor, FLAGS_catalog_service_port);
 
   if (EnableInternalSslConnections()) {
+    SSLProtocol ssl_version;
+    ABORT_IF_ERROR(
+        SSLProtoVersions::StringToProtocol(FLAGS_ssl_minimum_version, &ssl_version));
     LOG(INFO) << "Enabling SSL for CatalogService";
     builder.ssl(FLAGS_ssl_server_certificate, FLAGS_ssl_private_key)
         .pem_password_cmd(FLAGS_ssl_private_key_password_cmd)
+        .ssl_version(ssl_version)
         .cipher_list(FLAGS_ssl_cipher_list);
   }
   ABORT_IF_ERROR(builder.metrics(metrics.get()).Build(&server));

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/16ce201f/be/src/rpc/thrift-client.cc
----------------------------------------------------------------------
diff --git a/be/src/rpc/thrift-client.cc b/be/src/rpc/thrift-client.cc
index 6f01073..a64cdd3 100644
--- a/be/src/rpc/thrift-client.cc
+++ b/be/src/rpc/thrift-client.cc
@@ -23,6 +23,7 @@
 #include <thrift/Thrift.h>
 #include <gutil/strings/substitute.h>
 
+#include "util/network-util.h"
 #include "util/time.h"
 
 #include "common/names.h"
@@ -33,9 +34,22 @@ using namespace strings;
 
 DECLARE_string(ssl_client_ca_certificate);
 DECLARE_string(ssl_cipher_list);
+DECLARE_string(ssl_minimum_version);
 
 namespace impala {
 
+ThriftClientImpl::ThriftClientImpl(const std::string& ipaddress, int port, bool ssl)
+  : address_(MakeNetworkAddress(ipaddress, port)), ssl_(ssl) {
+  if (ssl_) {
+    SSLProtocol version;
+    socket_create_status_ =
+        SSLProtoVersions::StringToProtocol(FLAGS_ssl_minimum_version, &version);
+    if (!socket_create_status_.ok()) return;
+    ssl_factory_.reset(new TSSLSocketFactory(version));
+  }
+  socket_create_status_ = CreateSocket();
+}
+
 Status ThriftClientImpl::Open() {
   if (!socket_create_status_.ok()) return socket_create_status_;
   try {

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/16ce201f/be/src/rpc/thrift-client.h
----------------------------------------------------------------------
diff --git a/be/src/rpc/thrift-client.h b/be/src/rpc/thrift-client.h
index 26982a1..778075c 100644
--- a/be/src/rpc/thrift-client.h
+++ b/be/src/rpc/thrift-client.h
@@ -71,11 +71,7 @@ class ThriftClientImpl {
   Status socket_create_status() { return socket_create_status_; }
 
  protected:
-  ThriftClientImpl(const std::string& ipaddress, int port, bool ssl)
-      : address_(MakeNetworkAddress(ipaddress, port)), ssl_(ssl) {
-    if (ssl_) ssl_factory_.reset(new TSSLSocketFactory());
-    socket_create_status_ = CreateSocket();
-  }
+  ThriftClientImpl(const std::string& ipaddress, int port, bool ssl);
 
   /// Create a new socket without opening it. Returns an error if the socket could not
   /// be created.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/16ce201f/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 a7c5ca5..67e149a 100644
--- a/be/src/rpc/thrift-server-test.cc
+++ b/be/src/rpc/thrift-server-test.cc
@@ -30,9 +30,11 @@
 using namespace impala;
 using namespace strings;
 using namespace apache::thrift;
+using apache::thrift::transport::SSLProtocol;
 
 DECLARE_string(ssl_client_ca_certificate);
 DECLARE_string(ssl_cipher_list);
+DECLARE_string(ssl_minimum_version);
 
 DECLARE_int32(state_store_port);
 
@@ -257,6 +259,92 @@ TEST(SslTest, MismatchedCiphers) {
       TTransportException);
 }
 
+// Test that StringToProtocol() correctly maps strings to their symbolic protocol
+// 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}};
+  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) {
+  auto flag =
+      ScopedFlagSetter<string>::Make(&FLAGS_ssl_client_ca_certificate, SERVER_CERT);
+
+  // A config is really a pair (server_version, whitelist), where 'server_version' is the
+  // server TLS version to test, and 'whitelist' is the set of client protocols that
+  // should be able to connect successfully. This test tries all client protocols,
+  // expecting those in the whitelist to succeed, and those that are not to fail.
+  struct Config {
+    SSLProtocol server_version;
+    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
+  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}},
+      {TLSv1_1, {TLSv1_1_plus, TLSv1_1, TLSv1_0_plus}},
+      {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
+    // connect a client to it with every possible spec. This is an N^2 test, but the value
+    // of N is 6.
+    int port = GetServerPort();
+
+    ThriftServer* server;
+    EXPECT_OK(ThriftServerBuilder("DummyStatestore", MakeProcessor(), port)
+                  .ssl(SERVER_CERT, PRIVATE_KEY)
+                  .ssl_version(config.server_version)
+                  .Build(&server));
+    EXPECT_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);
+      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),
+            TTransportException)
+            << "TLS version: " << config.server_version
+            << ", client version: " << client_version.first;
+      } else {
+        EXPECT_NO_THROW({
+          ssl_client.iface()->RegisterSubscriber(
+              resp, TRegisterSubscriberRequest(), &send_done);
+        }) << "TLS version: "
+           << config.server_version << ", client version: " << client_version.first;
+      }
+    }
+  }
+}
+
 TEST(SslTest, MatchedCiphers) {
   int port = GetServerPort();
   ThriftServer* server;

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/16ce201f/be/src/rpc/thrift-server.cc
----------------------------------------------------------------------
diff --git a/be/src/rpc/thrift-server.cc b/be/src/rpc/thrift-server.cc
index e1e9a7f..1a558d9 100644
--- a/be/src/rpc/thrift-server.cc
+++ b/be/src/rpc/thrift-server.cc
@@ -36,10 +36,12 @@
 #include <sstream>
 #include "gen-cpp/Types_types.h"
 #include "rpc/TAcceptQueueServer.h"
+#include "rpc/auth-provider.h"
 #include "rpc/authentication.h"
 #include "rpc/thrift-server.h"
 #include "rpc/thrift-thread.h"
 #include "util/debug-util.h"
+#include "util/metrics.h"
 #include "util/network-util.h"
 #include "util/os-util.h"
 #include "util/uid-util.h"
@@ -58,8 +60,8 @@ using namespace apache::thrift::server;
 using namespace apache::thrift::transport;
 using namespace apache::thrift;
 
-DEFINE_int32(rpc_cnxn_attempts, 10, "Deprecated");
-DEFINE_int32(rpc_cnxn_retry_interval_ms, 2000, "Deprecated");
+DEFINE_int32_hidden(rpc_cnxn_attempts, 10, "Deprecated");
+DEFINE_int32_hidden(rpc_cnxn_retry_interval_ms, 2000, "Deprecated");
 
 DECLARE_bool(enable_accept_queue_server);
 DECLARE_string(principal);
@@ -70,6 +72,28 @@ DECLARE_string(ssl_cipher_list);
 
 namespace impala {
 
+// Specifies the allowed set of values for --ssl_minimum_version. To keep consistent with
+// Apache Kudu, specifying a single version enables all versions including and succeeding
+// 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}};
+
+Status SSLProtoVersions::StringToProtocol(const string& in, SSLProtocol* protocol) {
+  for (const auto& proto : SSLProtoVersions::PROTO_MAP) {
+    if (iequals(in, proto.first)) {
+      *protocol = proto.second;
+      return Status::OK();
+    }
+  }
+
+  return Status(Substitute("Unknown TLS version: '$0'", in));
+}
+
 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
@@ -328,11 +352,14 @@ ThriftServer::ThriftServer(const string& name,
   }
 }
 
+namespace {
+
 /// Factory subclass to override getPassword() which provides a password string to Thrift
 /// to decrypt the private key file.
 class ImpalaSslSocketFactory : public TSSLSocketFactory {
  public:
-  ImpalaSslSocketFactory(const string& password) : password_(password) { }
+  ImpalaSslSocketFactory(SSLProtocol version, const string& password)
+    : TSSLSocketFactory(version), password_(password) {}
 
  protected:
   virtual void getPassword(string& output, int size) {
@@ -344,13 +371,13 @@ class ImpalaSslSocketFactory : public TSSLSocketFactory {
   /// The password string.
   const string password_;
 };
-
+}
 Status ThriftServer::CreateSocket(boost::shared_ptr<TServerTransport>* socket) {
   if (ssl_enabled()) {
     // This 'factory' is only called once, since CreateSocket() is only called from
     // Start()
     boost::shared_ptr<TSSLSocketFactory> socket_factory(
-        new ImpalaSslSocketFactory(key_password_));
+        new ImpalaSslSocketFactory(version_, key_password_));
     socket_factory->overrideDefaultPasswordCallback();
     try {
       if (!cipher_list_.empty()) socket_factory->ciphers(cipher_list_);
@@ -367,8 +394,9 @@ Status ThriftServer::CreateSocket(boost::shared_ptr<TServerTransport>* socket) {
   }
 }
 
-Status ThriftServer::EnableSsl(const string& certificate, const string& private_key,
-    const string& pem_password_cmd, const std::string& ciphers) {
+Status ThriftServer::EnableSsl(SSLProtocol version, const string& certificate,
+    const string& private_key, const string& pem_password_cmd,
+    const std::string& ciphers) {
   DCHECK(!started_);
   if (certificate.empty()) return Status(TErrorCode::SSL_CERTIFICATE_PATH_BLANK);
   if (private_key.empty()) return Status(TErrorCode::SSL_PRIVATE_KEY_PATH_BLANK);
@@ -386,6 +414,7 @@ Status ThriftServer::EnableSsl(const string& certificate, const string& private_
   certificate_path_ = certificate;
   private_key_path_ = private_key;
   cipher_list_ = ciphers;
+  version_ = version;
 
   if (!pem_password_cmd.empty()) {
     if (!RunShellProcess(pem_password_cmd, &key_password_, true)) {

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/16ce201f/be/src/rpc/thrift-server.h
----------------------------------------------------------------------
diff --git a/be/src/rpc/thrift-server.h b/be/src/rpc/thrift-server.h
index e52edea..e486e37 100644
--- a/be/src/rpc/thrift-server.h
+++ b/be/src/rpc/thrift-server.h
@@ -18,21 +18,23 @@
 #ifndef IMPALA_RPC_THRIFT_SERVER_H
 #define IMPALA_RPC_THRIFT_SERVER_H
 
-#include <boost/thread/mutex.hpp>
 #include <boost/scoped_ptr.hpp>
 #include <boost/shared_ptr.hpp>
+#include <boost/thread/mutex.hpp>
 #include <boost/unordered_map.hpp>
 #include <boost/uuid/uuid_generators.hpp>
-#include <thrift/server/TServer.h>
 #include <thrift/TProcessor.h>
+#include <thrift/server/TServer.h>
+#include <thrift/transport/TSSLSocket.h>
 
 #include "common/status.h"
-#include "rpc/auth-provider.h"
 #include "util/metrics.h"
 #include "util/thread.h"
 
 namespace impala {
 
+class AuthProvider;
+
 /// Utility class for all Thrift servers. Runs a threaded server by default, or a
 /// TThreadPoolServer with, by default, 2 worker threads, that exposes the interface
 /// described by a user-supplied TProcessor object.
@@ -156,13 +158,15 @@ class ThriftServer {
       AuthProvider* auth_provider = nullptr, MetricGroup* metrics = nullptr,
       int num_worker_threads = DEFAULT_WORKER_THREADS, ServerType server_type = Threaded);
 
-  /// Enables secure access over SSL. Must be called before Start(). The first two
-  /// arguments are paths to certificate and private key files in .PEM format,
-  /// respectively. If either file does not exist, an error is returned. The final
-  /// optional argument provides the command to run if a password is required to decrypt
-  /// the private key. It is invoked once, and the resulting password is used only for
-  /// password-protected .PEM files.
-  Status EnableSsl(const std::string& certificate, const std::string& private_key,
+  /// Enables secure access over SSL. Must be called before Start(). The first three
+  /// arguments are the minimum SSL/TLS version, and paths to certificate and private key
+  /// files in .PEM format, respectively. If either file does not exist, an error is
+  /// returned. The fourth, optional, argument provides the command to run if a password
+  /// is required to decrypt the private key. It is invoked once, and the resulting
+  /// password is used only for password-protected .PEM files. The final argument is a
+  /// string containing a list of cipher suites, separated by commas, to enable.
+  Status EnableSsl(apache::thrift::transport::SSLProtocol version,
+      const std::string& certificate, const std::string& private_key,
       const std::string& pem_password_cmd = "", const std::string& ciphers = "");
 
   /// Creates the server socket on which this server listens. May be SSL enabled. Returns
@@ -191,6 +195,9 @@ class ThriftServer {
   /// List of ciphers that are ok for clients to use when connecting.
   std::string cipher_list_;
 
+  /// The SSL/TLS protocol client versions that this server will allow to connect.
+  apache::thrift::transport::SSLProtocol version_;
+
   /// How many worker threads to use to serve incoming requests
   /// (requests are queued if no thread is immediately available)
   int num_worker_threads_;
@@ -286,6 +293,12 @@ class ThriftServerBuilder {
     return *this;
   }
 
+  /// Sets the SSL/TLS client version(s) that this server will allow to connect.
+  ThriftServerBuilder& ssl_version(apache::thrift::transport::SSLProtocol version) {
+    version_ = version;
+    return *this;
+  }
+
   /// Sets the command used to compute the password for the SSL private key. Default is
   /// empty, i.e. no password needed.
   ThriftServerBuilder& pem_password_cmd(const std::string& pem_password_cmd) {
@@ -308,8 +321,8 @@ class ThriftServerBuilder {
     std::unique_ptr<ThriftServer> ptr(new ThriftServer(name_, processor_, port_,
         auth_provider_, metrics_, num_worker_threads_, server_type_));
     if (enable_ssl_) {
-      RETURN_IF_ERROR(
-          ptr->EnableSsl(certificate_, private_key_, pem_password_cmd_, ciphers_));
+      RETURN_IF_ERROR(ptr->EnableSsl(
+          version_, certificate_, private_key_, pem_password_cmd_, ciphers_));
     }
     (*server) = ptr.release();
     return Status::OK();
@@ -326,12 +339,24 @@ class ThriftServerBuilder {
   MetricGroup* metrics_ = nullptr;
 
   bool enable_ssl_ = false;
+  apache::thrift::transport::SSLProtocol version_ =
+      apache::thrift::transport::SSLProtocol::TLSv1_0;
   std::string certificate_;
   std::string private_key_;
   std::string pem_password_cmd_;
   std::string ciphers_;
 };
 
+/// Contains a map from string for --ssl_minimum_version to Thrift's SSLProtocol.
+struct SSLProtoVersions {
+  static std::map<std::string, apache::thrift::transport::SSLProtocol> PROTO_MAP;
+
+  /// Given a string, find a corresponding SSLProtocol from PROTO_MAP. Returns an error if
+  /// one cannot be found. Matching is case-insensitive.
+  static Status StringToProtocol(
+      const std::string& in, apache::thrift::transport::SSLProtocol* protocol);
+};
+
 // Returns true if, per the process configuration flags, server<->server communications
 // should use SSL.
 bool EnableInternalSslConnections();

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/16ce201f/be/src/service/impala-server.cc
----------------------------------------------------------------------
diff --git a/be/src/service/impala-server.cc b/be/src/service/impala-server.cc
index 5778041..00a9d9a 100644
--- a/be/src/service/impala-server.cc
+++ b/be/src/service/impala-server.cc
@@ -96,6 +96,7 @@ using boost::system_time;
 using boost::uuids::random_generator;
 using boost::uuids::uuid;
 using namespace apache::thrift;
+using namespace apache::thrift::transport;
 using namespace boost::posix_time;
 using namespace beeswax;
 using namespace rapidjson;
@@ -178,6 +179,15 @@ DEFINE_string(ssl_cipher_list, "",
     "ciphers for more information. If empty, the default cipher list for your platform "
     "is used");
 
+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
+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"
     " for before it is closed (and all running queries cancelled) by Impala. If 0, idle"
     " sessions are never expired.");
@@ -1933,6 +1943,12 @@ Status CreateImpalaServer(ExecEnv* exec_env, int beeswax_port, int hs2_port, int
 
   impala_server->reset(new ImpalaServer(exec_env));
 
+  SSLProtocol ssl_version = SSLProtocol::TLSv1_0;
+  if (!FLAGS_ssl_server_certificate.empty() || EnableInternalSslConnections()) {
+    RETURN_IF_ERROR(
+        SSLProtoVersions::StringToProtocol(FLAGS_ssl_minimum_version, &ssl_version));
+  }
+
   if (be_port != 0 && be_server != nullptr) {
     boost::shared_ptr<ImpalaInternalService> thrift_if(new ImpalaInternalService());
     boost::shared_ptr<TProcessor> be_processor(
@@ -1947,6 +1963,7 @@ Status CreateImpalaServer(ExecEnv* exec_env, int beeswax_port, int hs2_port, int
       LOG(INFO) << "Enabling SSL for backend";
       be_builder.ssl(FLAGS_ssl_server_certificate, FLAGS_ssl_private_key)
           .pem_password_cmd(FLAGS_ssl_private_key_password_cmd)
+          .ssl_version(ssl_version)
           .cipher_list(FLAGS_ssl_cipher_list);
     }
     RETURN_IF_ERROR(be_builder.metrics(exec_env->metrics()).Build(be_server));
@@ -1974,6 +1991,7 @@ Status CreateImpalaServer(ExecEnv* exec_env, int beeswax_port, int hs2_port, int
       LOG(INFO) << "Enabling SSL for Beeswax";
       builder.ssl(FLAGS_ssl_server_certificate, FLAGS_ssl_private_key)
           .pem_password_cmd(FLAGS_ssl_private_key_password_cmd)
+          .ssl_version(ssl_version)
           .cipher_list(FLAGS_ssl_cipher_list);
     }
     RETURN_IF_ERROR(
@@ -2000,6 +2018,7 @@ Status CreateImpalaServer(ExecEnv* exec_env, int beeswax_port, int hs2_port, int
       LOG(INFO) << "Enabling SSL for HiveServer2";
       builder.ssl(FLAGS_ssl_server_certificate, FLAGS_ssl_private_key)
           .pem_password_cmd(FLAGS_ssl_private_key_password_cmd)
+          .ssl_version(ssl_version)
           .cipher_list(FLAGS_ssl_cipher_list);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/16ce201f/be/src/statestore/statestore-subscriber.cc
----------------------------------------------------------------------
diff --git a/be/src/statestore/statestore-subscriber.cc b/be/src/statestore/statestore-subscriber.cc
index 69ddfdc..7cfcaf7 100644
--- a/be/src/statestore/statestore-subscriber.cc
+++ b/be/src/statestore/statestore-subscriber.cc
@@ -39,6 +39,7 @@
 
 using boost::posix_time::seconds;
 using namespace apache::thrift;
+using namespace apache::thrift::transport;
 using namespace strings;
 
 DEFINE_int32(statestore_subscriber_timeout_seconds, 30, "The amount of time (in seconds)"
@@ -53,6 +54,7 @@ DECLARE_string(ssl_server_certificate);
 DECLARE_string(ssl_private_key);
 DECLARE_string(ssl_private_key_password_cmd);
 DECLARE_string(ssl_cipher_list);
+DECLARE_string(ssl_minimum_version);
 
 namespace impala {
 
@@ -196,9 +198,13 @@ Status StatestoreSubscriber::Start() {
     ThriftServerBuilder builder(
         "StatestoreSubscriber", processor, heartbeat_address_.port);
     if (EnableInternalSslConnections()) {
+      SSLProtocol ssl_version;
+      RETURN_IF_ERROR(
+          SSLProtoVersions::StringToProtocol(FLAGS_ssl_minimum_version, &ssl_version));
       LOG(INFO) << "Enabling SSL for Statestore subscriber";
       builder.ssl(FLAGS_ssl_server_certificate, FLAGS_ssl_private_key)
           .pem_password_cmd(FLAGS_ssl_private_key_password_cmd)
+          .ssl_version(ssl_version)
           .cipher_list(FLAGS_ssl_cipher_list);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/16ce201f/be/src/statestore/statestored-main.cc
----------------------------------------------------------------------
diff --git a/be/src/statestore/statestored-main.cc b/be/src/statestore/statestored-main.cc
index f4c4672..1dcb682 100644
--- a/be/src/statestore/statestored-main.cc
+++ b/be/src/statestore/statestored-main.cc
@@ -43,11 +43,13 @@ DECLARE_string(ssl_server_certificate);
 DECLARE_string(ssl_private_key);
 DECLARE_string(ssl_private_key_password_cmd);
 DECLARE_string(ssl_cipher_list);
+DECLARE_string(ssl_minimum_version);
 
 #include "common/names.h"
 
 using namespace impala;
 using namespace apache::thrift;
+using namespace apache::thrift::transport;
 
 int StatestoredMain(int argc, char** argv) {
   // Override default for webserver port
@@ -87,9 +89,13 @@ int StatestoredMain(int argc, char** argv) {
   ThriftServer* server;
   ThriftServerBuilder builder("StatestoreService", processor, FLAGS_state_store_port);
   if (EnableInternalSslConnections()) {
+    SSLProtocol ssl_version;
+    ABORT_IF_ERROR(
+        SSLProtoVersions::StringToProtocol(FLAGS_ssl_minimum_version, &ssl_version));
     LOG(INFO) << "Enabling SSL for Statestore";
     builder.ssl(FLAGS_ssl_server_certificate, FLAGS_ssl_private_key)
         .pem_password_cmd(FLAGS_ssl_private_key_password_cmd)
+        .ssl_version(ssl_version)
         .cipher_list(FLAGS_ssl_cipher_list);
   }
   ABORT_IF_ERROR(builder.metrics(metrics.get()).Build(&server));

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/16ce201f/bin/impala-config.sh
----------------------------------------------------------------------
diff --git a/bin/impala-config.sh b/bin/impala-config.sh
index 5470bb0..58b8529 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=447-f3bee11cc6
+export IMPALA_TOOLCHAIN_BUILD_ID=450-fc9954b4fa
 # 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-p9
+export IMPALA_THRIFT_VERSION=0.9.0-p10
 export IMPALA_THRIFT_JAVA_VERSION=0.9.0
 export IMPALA_ZLIB_VERSION=1.2.8