You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by jo...@apache.org on 2022/08/23 16:02:55 UTC

[impala] 01/03: IMPALA-10069: Support TLS 1.3 ciphersuites

This is an automated email from the ASF dual-hosted git repository.

joemcdonnell pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/impala.git

commit e4a98d81245c3dbca56575c28940aa973ec20a48
Author: Joe McDonnell <jo...@cloudera.com>
AuthorDate: Wed Feb 24 12:43:31 2021 -0800

    IMPALA-10069: Support TLS 1.3 ciphersuites
    
    OpenSSL 1.1.1 adds support for TLS 1.3. TLS 1.3 has a new
    set of cipher suites that are maintained separately from
    the TLS 1.2 ciphers. This caused test failures on tests
    that expected failures when setting invalid TLS 1.2 ciphers.
    It also rendered some success test cases invalid, because
    the TLS 1.3 ciphers would work even if TLS 1.2 didn't.
    
    This adds the tls_ciphersuites startup parameter, which
    customizes the TLS 1.3 cipher suites. tls_ciphersuites is
    only effective when Impala is built for a system with OpenSSL
    1.1.1 or above.
    
    This uses tls_ciphersuites to fix the existing TLS 1.2 cipher
    tests. It also adds a set of tests for TLS 1.3 cipher suites.
    
    KRPC and the webserver now support ssl_minimum_version=tlsv1.3.
    However, Thrift does not support this configuration yet, so
    this is not supported for impalad yet. To support TLS 1.3 tests
    on Thrift, this adds a disable_tls12 option to ThriftServer
    and ThriftClient. This will be removed when
    ssl_minimum_version=tlsv1.3 is supported.
    
    Testing:
     - Ran the backend tests with TLS checks (rpc-mgr-test, rpc-mgr-kerberized-test,
       webserver-test, and thrift-server-test) on Ubuntu 18 and Ubuntu 20
     - Added tests for tls_ciphersuites in rpc-mgr-test and thrift-server-test
     - Ran a core test on Centos 7
     - Ran a core test on Ubuntu 16
    
    Change-Id: I6974dae7fb429599847165614adc4eaaf338f744
    Reviewed-on: http://gerrit.cloudera.org:8080/18316
    Reviewed-by: Wenzhe Zhou <wz...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 be/src/rpc/rpc-mgr-test.cc            |  74 ++++++++++++-
 be/src/rpc/rpc-mgr-test.h             |  15 ++-
 be/src/rpc/rpc-mgr.cc                 |   7 ++
 be/src/rpc/thrift-client.cc           |  13 ++-
 be/src/rpc/thrift-client.h            |  18 +++-
 be/src/rpc/thrift-server-test.cc      | 196 ++++++++++++++++++++++++++++++++--
 be/src/rpc/thrift-server.cc           |  63 +++--------
 be/src/rpc/thrift-server.h            |  39 ++++++-
 be/src/rpc/thrift-util.cc             |  90 ++++++++++++++++
 be/src/rpc/thrift-util.h              |  14 +++
 be/src/service/impala-server.cc       |   7 ++
 be/src/thirdparty/squeasel/squeasel.c |  24 ++++-
 be/src/util/webserver-test.cc         |  44 +++++++-
 be/src/util/webserver.cc              |   3 +
 14 files changed, 523 insertions(+), 84 deletions(-)

diff --git a/be/src/rpc/rpc-mgr-test.cc b/be/src/rpc/rpc-mgr-test.cc
index 242b6b837..792eefac1 100644
--- a/be/src/rpc/rpc-mgr-test.cc
+++ b/be/src/rpc/rpc-mgr-test.cc
@@ -117,10 +117,9 @@ TEST_P(RpcMgrTest, CorrectPasswordTls) {
   tls_rpc_mgr.Shutdown();
 }
 
-#ifndef __aarch64__
 // Test with a bad TLS cipher and verify that an error is thrown.
 TEST_P(RpcMgrTest, BadCiphersTls) {
-  ScopedSetTlsFlags s(SERVER_CERT, PRIVATE_KEY, SERVER_CERT, "", "not_a_cipher");
+  ScopedSetTlsFlags s(SERVER_CERT, PRIVATE_KEY, SERVER_CERT, "", "not_a_cipher", "");
 
   RpcMgr tls_rpc_mgr(IsInternalTlsConfigured());
   NetworkAddressPB tls_krpc_address;
@@ -134,12 +133,11 @@ TEST_P(RpcMgrTest, BadCiphersTls) {
   ASSERT_FALSE(tls_rpc_mgr.Init(tls_krpc_address).ok());
   tls_rpc_mgr.Shutdown();
 }
-#endif
 
 // Test with a valid TLS cipher.
 TEST_P(RpcMgrTest, ValidCiphersTls) {
   ScopedSetTlsFlags s(SERVER_CERT, PRIVATE_KEY, SERVER_CERT, "",
-      TLS1_0_COMPATIBLE_CIPHER);
+      TLS1_0_COMPATIBLE_CIPHER, "");
 
   RpcMgr tls_rpc_mgr(IsInternalTlsConfigured());
   NetworkAddressPB tls_krpc_address;
@@ -159,7 +157,7 @@ TEST_P(RpcMgrTest, ValidCiphersTls) {
 TEST_P(RpcMgrTest, ValidMultiCiphersTls) {
   const string cipher_list = Substitute("$0,$1", TLS1_0_COMPATIBLE_CIPHER,
       TLS1_0_COMPATIBLE_CIPHER_2);
-  ScopedSetTlsFlags s(SERVER_CERT, PRIVATE_KEY, SERVER_CERT, "", cipher_list);
+  ScopedSetTlsFlags s(SERVER_CERT, PRIVATE_KEY, SERVER_CERT, "", cipher_list, "");
 
   RpcMgr tls_rpc_mgr(IsInternalTlsConfigured());
   NetworkAddressPB tls_krpc_address;
@@ -175,6 +173,72 @@ TEST_P(RpcMgrTest, ValidMultiCiphersTls) {
   tls_rpc_mgr.Shutdown();
 }
 
+// TLS 1.3 tests only make sense on OpenSSL 1.1.1 and above
+// These tests set ssl_minimum_version="tlsv1.3". This is safe for these tests,
+// because KRPC supports a minimum version of tlsv1.3. It is not supported
+// in general.
+#if OPENSSL_VERSION_NUMBER >= 0x10101000L
+
+// Test with a bad TLS cipher and verify that an error is thrown.
+TEST_P(RpcMgrTest, BadTlsCiphersuites) {
+  ScopedSetTlsFlags s(SERVER_CERT, PRIVATE_KEY, SERVER_CERT, "", "",
+      "not_a_ciphersuite", "tlsv1.3");
+
+  RpcMgr tls_rpc_mgr(IsInternalTlsConfigured());
+  NetworkAddressPB tls_krpc_address;
+  IpAddr ip;
+  ASSERT_OK(HostnameToIpAddr(FLAGS_hostname, &ip));
+
+  int32_t tls_service_port = FindUnusedEphemeralPort();
+  tls_krpc_address = MakeNetworkAddressPB(ip, tls_service_port,
+      tls_rpc_mgr.GetUdsAddressUniqueId());
+
+  ASSERT_FALSE(tls_rpc_mgr.Init(tls_krpc_address).ok());
+  tls_rpc_mgr.Shutdown();
+}
+
+// Test with a valid TLS 1.3 ciphersuite.
+TEST_P(RpcMgrTest, ValidTlsCiphersuites) {
+  ScopedSetTlsFlags s(SERVER_CERT, PRIVATE_KEY, SERVER_CERT, "",
+      "", TLS1_3_CIPHERSUITE, "tlsv1.3");
+
+  RpcMgr tls_rpc_mgr(IsInternalTlsConfigured());
+  NetworkAddressPB tls_krpc_address;
+  IpAddr ip;
+  ASSERT_OK(HostnameToIpAddr(FLAGS_hostname, &ip));
+
+  int32_t tls_service_port = FindUnusedEphemeralPort();
+  tls_krpc_address = MakeNetworkAddressPB(ip, tls_service_port,
+      tls_rpc_mgr.GetUdsAddressUniqueId());
+
+  ASSERT_OK(tls_rpc_mgr.Init(tls_krpc_address));
+  ASSERT_OK(RunMultipleServicesTest(&tls_rpc_mgr, tls_krpc_address));
+  tls_rpc_mgr.Shutdown();
+}
+
+// Test with multiple valid TLS 1.3 ciphersuites
+TEST_P(RpcMgrTest, ValidMultiTlsCiphersuite) {
+  const string ciphersuite_list = Substitute("$0:$1", TLS1_3_CIPHERSUITE,
+      TLS1_3_CIPHERSUITE_2);
+  ScopedSetTlsFlags s(SERVER_CERT, PRIVATE_KEY, SERVER_CERT, "", "",
+      ciphersuite_list, "tlsv1.3");
+
+  RpcMgr tls_rpc_mgr(IsInternalTlsConfigured());
+  NetworkAddressPB tls_krpc_address;
+  IpAddr ip;
+  ASSERT_OK(HostnameToIpAddr(FLAGS_hostname, &ip));
+
+  int32_t tls_service_port = FindUnusedEphemeralPort();
+  tls_krpc_address = MakeNetworkAddressPB(ip, tls_service_port,
+      tls_rpc_mgr.GetUdsAddressUniqueId());
+
+  ASSERT_OK(tls_rpc_mgr.Init(tls_krpc_address));
+  ASSERT_OK(RunMultipleServicesTest(&tls_rpc_mgr, tls_krpc_address));
+  tls_rpc_mgr.Shutdown();
+}
+
+#endif // OPENSSL_VERSION_NUMBER
+
 // Test behavior with a slow service.
 TEST_P(RpcMgrTest, SlowCallback) {
   // Use a callback which is slow to respond.
diff --git a/be/src/rpc/rpc-mgr-test.h b/be/src/rpc/rpc-mgr-test.h
index 74309543b..d04024634 100644
--- a/be/src/rpc/rpc-mgr-test.h
+++ b/be/src/rpc/rpc-mgr-test.h
@@ -25,6 +25,7 @@
 #include "kudu/rpc/rpc_controller.h"
 #include "kudu/rpc/rpc_header.pb.h"
 #include "kudu/rpc/rpc_sidecar.h"
+#include "kudu/security/security_flags.h"
 #include "kudu/util/status.h"
 #include "rpc/rpc-mgr.inline.h"
 #include "runtime/exec-env.h"
@@ -61,6 +62,8 @@ 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);
+DECLARE_string(tls_ciphersuites);
 
 // The path of the current executable file that is required for passing into the SASL
 // library as the 'application name'.
@@ -86,12 +89,17 @@ const string& PASSWORD_PROTECTED_PRIVATE_KEY =
 class ScopedSetTlsFlags {
  public:
   ScopedSetTlsFlags(const string& cert, const string& pkey, const string& ca_cert,
-      const string& pkey_passwd = "", const string& ciphers = "") {
+      const string& pkey_passwd = "", const string& ciphers = "",
+      const string& tls_ciphersuites =
+          kudu::security::SecurityDefaults::kDefaultTlsCipherSuites,
+      const string& ssl_minimum_version = "tlsv1.2") {
     FLAGS_ssl_server_certificate = cert;
     FLAGS_ssl_private_key = pkey;
     FLAGS_ssl_client_ca_certificate = ca_cert;
     FLAGS_ssl_private_key_password_cmd = pkey_passwd;
     FLAGS_ssl_cipher_list = ciphers;
+    FLAGS_tls_ciphersuites = tls_ciphersuites;
+    FLAGS_ssl_minimum_version = ssl_minimum_version;
   }
 
   ~ScopedSetTlsFlags() {
@@ -100,6 +108,8 @@ class ScopedSetTlsFlags {
     FLAGS_ssl_client_ca_certificate = "";
     FLAGS_ssl_private_key_password_cmd = "";
     FLAGS_ssl_cipher_list = "";
+    FLAGS_tls_ciphersuites = kudu::security::SecurityDefaults::kDefaultTlsCipherSuites;
+    FLAGS_ssl_minimum_version = "tlsv1.2";
   }
 };
 
@@ -108,6 +118,9 @@ class ScopedSetTlsFlags {
 const string TLS1_0_COMPATIBLE_CIPHER = "AES128-SHA";
 const string TLS1_0_COMPATIBLE_CIPHER_2 = "AES256-SHA";
 
+const string TLS1_3_CIPHERSUITE = "TLS_AES_256_GCM_SHA384";
+const string TLS1_3_CIPHERSUITE_2 = "TLS_CHACHA20_POLY1305_SHA256";
+
 #define PAYLOAD_SIZE (4096)
 
 class RpcMgrTest : public testing::TestWithParam<bool> {
diff --git a/be/src/rpc/rpc-mgr.cc b/be/src/rpc/rpc-mgr.cc
index 926739519..bcbf9d74a 100644
--- a/be/src/rpc/rpc-mgr.cc
+++ b/be/src/rpc/rpc-mgr.cc
@@ -69,6 +69,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(tls_ciphersuites);
 DECLARE_string(ssl_minimum_version);
 
 DECLARE_int64(impala_slow_rpc_threshold_ms);
@@ -177,6 +178,12 @@ Status RpcMgr::Init(const NetworkAddressPB& address) {
     if (!FLAGS_ssl_cipher_list.empty()) {
       bld.set_rpc_tls_ciphers(FLAGS_ssl_cipher_list);
     }
+    bld.set_rpc_tls_ciphersuites(FLAGS_tls_ciphersuites);
+    // If there are no TLS 1.3 ciphersuites listed, then exclude TLS 1.3, as it
+    // cannot function.
+    if (FLAGS_tls_ciphersuites.empty()) {
+      bld.set_rpc_tls_excluded_protocols({"TLSv1.3"});
+    }
     bld.set_rpc_tls_min_protocol(FLAGS_ssl_minimum_version);
     bld.set_rpc_encryption("required");
     bld.enable_inbound_tls();
diff --git a/be/src/rpc/thrift-client.cc b/be/src/rpc/thrift-client.cc
index e0852eb06..3b1d7d0e2 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 "rpc/thrift-util.h"
 #include "util/network-util.h"
 #include "util/openssl-util.h"
 #include "util/time.h"
@@ -36,11 +37,14 @@ using namespace strings;
 DECLARE_string(ssl_client_ca_certificate);
 DECLARE_string(ssl_cipher_list);
 DECLARE_string(ssl_minimum_version);
+DECLARE_string(tls_ciphersuites);
 
 namespace impala {
 
-ThriftClientImpl::ThriftClientImpl(const std::string& ipaddress, int port, bool ssl)
-  : address_(MakeNetworkAddress(ipaddress, port)), ssl_(ssl) {
+ThriftClientImpl::ThriftClientImpl(const std::string& ipaddress, int port, bool ssl,
+    bool disable_tls12)
+  : address_(MakeNetworkAddress(ipaddress, port)), ssl_(ssl),
+    disable_tls12_(disable_tls12) {
   if (ssl_) {
     SSLProtocol version;
     init_status_ =
@@ -52,7 +56,7 @@ ThriftClientImpl::ThriftClientImpl(const std::string& ipaddress, int port, bool
       init_status_ = Status(err);
     }
     if (!init_status_.ok()) return;
-    ssl_factory_.reset(new TSSLSocketFactory(version));
+    ssl_factory_.reset(new ImpalaTlsSocketFactory(version));
   }
   init_status_ = CreateSocket();
 }
@@ -127,7 +131,8 @@ Status ThriftClientImpl::CreateSocket() {
     socket_.reset(new TSocket(address_.hostname, address_.port));
   } else {
     try {
-      if (!FLAGS_ssl_cipher_list.empty()) ssl_factory_->ciphers(FLAGS_ssl_cipher_list);
+      ssl_factory_->configureCiphers(FLAGS_ssl_cipher_list, FLAGS_tls_ciphersuites,
+          disable_tls12_);
       ssl_factory_->loadTrustedCertificates(FLAGS_ssl_client_ca_certificate.c_str());
       socket_ = ssl_factory_->createSocket(address_.hostname, address_.port);
     } catch (const TException& e) {
diff --git a/be/src/rpc/thrift-client.h b/be/src/rpc/thrift-client.h
index 60a13cb5a..9e3c8eac3 100644
--- a/be/src/rpc/thrift-client.h
+++ b/be/src/rpc/thrift-client.h
@@ -30,6 +30,7 @@
 #include "transport/TSasl.h"
 #include "rpc/authentication.h"
 #include "rpc/thrift-server.h"
+#include "rpc/thrift-util.h"
 #include "gen-cpp/Types_types.h"
 
 DECLARE_string(principal);
@@ -64,7 +65,7 @@ class ThriftClientImpl {
   Status init_status() { return init_status_; }
 
  protected:
-  ThriftClientImpl(const std::string& ipaddress, int port, bool ssl);
+  ThriftClientImpl(const std::string& ipaddress, int port, bool ssl, bool disable_tls12);
 
   /// Create a new socket without opening it. Returns an error if the socket could not
   /// be created.
@@ -76,6 +77,10 @@ class ThriftClientImpl {
   /// True if ssl encryption is enabled on this connection.
   bool ssl_;
 
+  /// Whether to disable TLSv1.2. This is used to test TLSv1.3 ciphersuites.
+  /// TODO: Remove this when Thrift supports ssl_minimum_version=tlsv1.3.
+  bool disable_tls12_;
+
   Status init_status_;
 
   /// Sasl Client object.  Contains client kerberos identification data.
@@ -85,7 +90,7 @@ class ThriftClientImpl {
   /// This factory sets up the openSSL library state and needs to be alive as long as its
   /// owner(a ThriftClientImpl instance) does. Otherwise the OpenSSL state is lost
   /// (refer IMPALA-2747).
-  boost::scoped_ptr<apache::thrift::transport::TSSLSocketFactory> ssl_factory_;
+  boost::scoped_ptr<ImpalaTlsSocketFactory> ssl_factory_;
 
   /// All shared pointers, because Thrift requires them to be
   std::shared_ptr<apache::thrift::transport::TSocket> socket_;
@@ -107,9 +112,11 @@ class ThriftClient : public ThriftClientImpl {
   ///  - auth_provider: Authentication scheme to use. If NULL, use the global default
   ///    client<->demon authentication scheme.
   ///  - ssl: if true, SSL is enabled on this connection
+  ///  - disable_tls12: If true, disable TLS 1.2. This is used for testing TLS 1.3.
+  ///    It can be removed when Thrift supports ssl_minimum_version=tlsv1.3.
   ThriftClient(const std::string& ipaddress, int port,
       const std::string& service_name = "", AuthProvider* auth_provider = NULL,
-      bool ssl = false);
+      bool ssl = false, bool disable_tls12 = false);
 
   /// Returns the object used to actually make RPCs against the remote server
   InterfaceType* iface() { return iface_.get(); }
@@ -122,8 +129,9 @@ class ThriftClient : public ThriftClientImpl {
 
 template <class InterfaceType>
 ThriftClient<InterfaceType>::ThriftClient(const std::string& ipaddress, int port,
-    const std::string& service_name, AuthProvider* auth_provider, bool ssl)
-    : ThriftClientImpl(ipaddress, port, ssl),
+    const std::string& service_name, AuthProvider* auth_provider, bool ssl,
+    bool disable_tls12)
+  : ThriftClientImpl(ipaddress, port, ssl, disable_tls12),
       iface_(new InterfaceType(protocol_)),
       auth_provider_(auth_provider) {
 
diff --git a/be/src/rpc/thrift-server-test.cc b/be/src/rpc/thrift-server-test.cc
index 873ee647f..d5070b70f 100644
--- a/be/src/rpc/thrift-server-test.cc
+++ b/be/src/rpc/thrift-server-test.cc
@@ -41,6 +41,7 @@ DECLARE_string(keytab_file);
 DECLARE_string(ssl_client_ca_certificate);
 DECLARE_string(ssl_cipher_list);
 DECLARE_string(ssl_minimum_version);
+DECLARE_string(tls_ciphersuites);
 
 DECLARE_int32(state_store_port);
 
@@ -68,6 +69,9 @@ static string principal_kt_path;
 static const string TLS1_0_COMPATIBLE_CIPHER = "AES128-SHA";
 static const string TLS1_0_COMPATIBLE_CIPHER_2 = "AES256-SHA";
 
+static const string TLS1_3_CIPHERSUITE = "TLS_AES_256_GCM_SHA384";
+static const string TLS1_3_CIPHERSUITE_2 = "TLS_CHACHA20_POLY1305_SHA256";
+
 /// Dummy server class (chosen because it has the smallest interface to implement) that
 /// tests can use to start Thrift servers.
 class DummyStatestoreService : public StatestoreServiceIf {
@@ -273,32 +277,31 @@ TEST(SslTest, ClientBeforeServer) {
 
 TEST(SslTest, BadCiphers) {
   int port = GetServerPort();
-#ifndef __aarch64__
   {
     ThriftServer* server;
     EXPECT_OK(ThriftServerBuilder("DummyStatestore", MakeProcessor(), port)
                   .ssl(SERVER_CERT, PRIVATE_KEY)
                   .cipher_list("this_is_not_a_cipher")
+                  .tls_ciphersuites("")
                   .Build(&server));
     EXPECT_FALSE(server->Start().ok());
   }
-#endif
   {
     ThriftServer* server;
     EXPECT_OK(ThriftServerBuilder("DummyStatestore", MakeProcessor(), port)
                   .ssl(SERVER_CERT, PRIVATE_KEY)
                   .Build(&server));
     EXPECT_OK(server->Start());
-#ifndef __aarch64__
     auto s1 =
         ScopedFlagSetter<string>::Make(&FLAGS_ssl_cipher_list, "this_is_not_a_cipher");
     auto s2 =
+        ScopedFlagSetter<string>::Make(&FLAGS_tls_ciphersuites, "");
+    auto s3 =
         ScopedFlagSetter<string>::Make(&FLAGS_ssl_client_ca_certificate, SERVER_CERT);
 
     ThriftClient<StatestoreServiceClientWrapper> ssl_client(
         "localhost", port, "", nullptr, true);
     EXPECT_FALSE(ssl_client.Open().ok());
-#endif
   }
 }
 
@@ -311,23 +314,24 @@ TEST(SslTest, MismatchedCiphers) {
                 .ssl(SERVER_CERT, PASSWORD_PROTECTED_PRIVATE_KEY)
                 .pem_password_cmd("echo password")
                 .cipher_list(TLS1_0_COMPATIBLE_CIPHER)
+                .tls_ciphersuites("")
                 .Build(&server));
   EXPECT_OK(server->Start());
-  auto s =
+  auto s1 =
       ScopedFlagSetter<string>::Make(&FLAGS_ssl_cipher_list, TLS1_0_COMPATIBLE_CIPHER_2);
+  auto s2 =
+      ScopedFlagSetter<string>::Make(&FLAGS_tls_ciphersuites, "");
   ThriftClient<StatestoreServiceClientWrapper> ssl_client(
       "localhost", port, "", nullptr, true);
 
   // Failure to negotiate a cipher will show up when data is sent, not when socket is
   // opened.
   EXPECT_OK(ssl_client.Open());
-#ifndef __aarch64__
   bool send_done = false;
   TRegisterSubscriberResponse resp;
   EXPECT_THROW(ssl_client.iface()->RegisterSubscriber(
                    resp, TRegisterSubscriberRequest(), &send_done),
       TTransportException);
-#endif
 }
 
 // Test that StringToProtocol() correctly maps strings to their symbolic protocol
@@ -415,12 +419,15 @@ TEST(SslTest, MatchedCiphers) {
                 .ssl(SERVER_CERT, PASSWORD_PROTECTED_PRIVATE_KEY)
                 .pem_password_cmd("echo password")
                 .cipher_list(TLS1_0_COMPATIBLE_CIPHER)
+                .tls_ciphersuites("")
                 .Build(&server));
   EXPECT_OK(server->Start());
 
   FLAGS_ssl_client_ca_certificate = SERVER_CERT;
-  auto s =
+  auto s1 =
       ScopedFlagSetter<string>::Make(&FLAGS_ssl_cipher_list, TLS1_0_COMPATIBLE_CIPHER);
+  auto s2 =
+      ScopedFlagSetter<string>::Make(&FLAGS_tls_ciphersuites, "");
   ThriftClient<StatestoreServiceClientWrapper> ssl_client(
       "localhost", port, "", nullptr, true);
 
@@ -443,15 +450,184 @@ TEST(SslTest, OverlappingMatchedCiphers) {
       .ssl(SERVER_CERT, PASSWORD_PROTECTED_PRIVATE_KEY)
       .pem_password_cmd("echo password")
       .cipher_list(CIPHER_LIST)
+      .tls_ciphersuites("")
       .Build(&server));
   EXPECT_OK(server->Start());
 
   FLAGS_ssl_client_ca_certificate = SERVER_CERT;
-  auto s = ScopedFlagSetter<string>::Make(&FLAGS_ssl_cipher_list,
+  auto s1 = ScopedFlagSetter<string>::Make(&FLAGS_ssl_cipher_list,
       Substitute("$0,not-a-cipher", TLS1_0_COMPATIBLE_CIPHER));
+  auto s2 = ScopedFlagSetter<string>::Make(&FLAGS_tls_ciphersuites, "");
+  ThriftClient<StatestoreServiceClientWrapper> ssl_client(
+      "localhost", port, "", nullptr, true);
+
+  EXPECT_OK(ssl_client.Open());
+
+  bool send_done = false;
+  TRegisterSubscriberResponse resp;
+  EXPECT_NO_THROW({
+        ssl_client.iface()->RegisterSubscriber(
+            resp, TRegisterSubscriberRequest(), &send_done);
+      });
+}
+
+#if OPENSSL_VERSION_NUMBER >= 0x10101000L
+TEST(SslTest, BadTlsCipherSuites) {
+  int port = GetServerPort();
+  {
+    ThriftServer* server;
+    EXPECT_OK(ThriftServerBuilder("DummyStatestore", MakeProcessor(), port)
+                  .ssl(SERVER_CERT, PRIVATE_KEY)
+                  .disable_tls12(true)
+                  .tls_ciphersuites("this_is_not_a_ciphersuite")
+                  .Build(&server));
+    EXPECT_FALSE(server->Start().ok());
+  }
+  {
+    ThriftServer* server;
+    EXPECT_OK(ThriftServerBuilder("DummyStatestore", MakeProcessor(), port)
+                  .ssl(SERVER_CERT, PRIVATE_KEY)
+                  .Build(&server));
+    EXPECT_OK(server->Start());
+    auto s1 = ScopedFlagSetter<string>::Make(&FLAGS_tls_ciphersuites,
+        "this_is_not_a_ciphersuite");
+    auto s2 =
+        ScopedFlagSetter<string>::Make(&FLAGS_ssl_client_ca_certificate, SERVER_CERT);
+
+    ThriftClient<StatestoreServiceClientWrapper> ssl_client(
+        "localhost", port, "", nullptr, true, true /* disable_tls12 */);
+    EXPECT_FALSE(ssl_client.Open().ok());
+  }
+}
+
+TEST(SslTest, MismatchedTlsCiphersuites) {
+  int port = GetServerPort();
+  FLAGS_ssl_client_ca_certificate = SERVER_CERT;
+
+  ThriftServer* server;
+  EXPECT_OK(ThriftServerBuilder("DummyStatestore", MakeProcessor(), port)
+                .ssl(SERVER_CERT, PASSWORD_PROTECTED_PRIVATE_KEY)
+                .pem_password_cmd("echo password")
+                .disable_tls12(true)
+                .tls_ciphersuites(TLS1_3_CIPHERSUITE)
+                .Build(&server));
+  EXPECT_OK(server->Start());
+  auto s =
+      ScopedFlagSetter<string>::Make(&FLAGS_tls_ciphersuites, TLS1_3_CIPHERSUITE_2);
+  ThriftClient<StatestoreServiceClientWrapper> ssl_client(
+      "localhost", port, "", nullptr, true, true /* disable_tls12 */);
+
+  // Failure to negotiate a cipher will show up when data is sent, not when socket is
+  // opened.
+  EXPECT_OK(ssl_client.Open());
+  bool send_done = false;
+  TRegisterSubscriberResponse resp;
+  EXPECT_THROW(ssl_client.iface()->RegisterSubscriber(
+                   resp, TRegisterSubscriberRequest(), &send_done),
+      TTransportException);
+}
+
+TEST(SslTest, MismatchTls12ServerTls13Client) {
+  int port = GetServerPort();
+  FLAGS_ssl_client_ca_certificate = SERVER_CERT;
+
+  // Disable TLS 1.3 on server by setting tls_ciphersuites="", allow TLS 1.2
+  ThriftServer* server;
+  EXPECT_OK(ThriftServerBuilder("DummyStatestore", MakeProcessor(), port)
+                .ssl(SERVER_CERT, PASSWORD_PROTECTED_PRIVATE_KEY)
+                .pem_password_cmd("echo password")
+                .tls_ciphersuites("")
+                .Build(&server));
+  EXPECT_OK(server->Start());
+
+  // Disable TLS 1.2 on client, allow TLS 1.3
+  ThriftClient<StatestoreServiceClientWrapper> ssl_client(
+      "localhost", port, "", nullptr, true, true /* disable_tls12 */);
+
+  // Failure to negotiate a cipher will show up when data is sent, not when socket is
+  // opened.
+  EXPECT_OK(ssl_client.Open());
+  bool send_done = false;
+  TRegisterSubscriberResponse resp;
+  EXPECT_THROW(ssl_client.iface()->RegisterSubscriber(
+                   resp, TRegisterSubscriberRequest(), &send_done),
+      TTransportException);
+}
+
+TEST(SslTest, MismatchTls13ServerTls12Client) {
+  int port = GetServerPort();
+  FLAGS_ssl_client_ca_certificate = SERVER_CERT;
+
+  // Disable TLS 1.2 on server, allow TLS 1.3
+  ThriftServer* server;
+  EXPECT_OK(ThriftServerBuilder("DummyStatestore", MakeProcessor(), port)
+                .ssl(SERVER_CERT, PASSWORD_PROTECTED_PRIVATE_KEY)
+                .pem_password_cmd("echo password")
+                .disable_tls12(true)
+                .Build(&server));
+  EXPECT_OK(server->Start());
+
+  // Disable TLS 1.3 on client by setting tls_ciphersuites="", allow TLS 1.2
+  auto s1 =
+      ScopedFlagSetter<string>::Make(&FLAGS_tls_ciphersuites, "");
   ThriftClient<StatestoreServiceClientWrapper> ssl_client(
       "localhost", port, "", nullptr, true);
 
+  // Failure to negotiate a cipher will show up when data is sent, not when socket is
+  // opened.
+  EXPECT_OK(ssl_client.Open());
+  bool send_done = false;
+  TRegisterSubscriberResponse resp;
+  EXPECT_THROW(ssl_client.iface()->RegisterSubscriber(
+                   resp, TRegisterSubscriberRequest(), &send_done),
+      TTransportException);
+}
+
+TEST(SslTest, MatchedTlsCiphersuites) {
+  int port = GetServerPort();
+  ThriftServer* server;
+  EXPECT_OK(ThriftServerBuilder("DummyStatestore", MakeProcessor(), port)
+                .ssl(SERVER_CERT, PASSWORD_PROTECTED_PRIVATE_KEY)
+                .pem_password_cmd("echo password")
+                .disable_tls12(true)
+                .tls_ciphersuites(TLS1_3_CIPHERSUITE)
+                .Build(&server));
+  EXPECT_OK(server->Start());
+
+  FLAGS_ssl_client_ca_certificate = SERVER_CERT;
+  auto s =
+      ScopedFlagSetter<string>::Make(&FLAGS_tls_ciphersuites, TLS1_3_CIPHERSUITE);
+  ThriftClient<StatestoreServiceClientWrapper> ssl_client(
+      "localhost", port, "", nullptr, true, true /* disable_tls12 */);
+
+  EXPECT_OK(ssl_client.Open());
+
+  bool send_done = false;
+  TRegisterSubscriberResponse resp;
+  EXPECT_NO_THROW({
+    ssl_client.iface()->RegisterSubscriber(
+        resp, TRegisterSubscriberRequest(), &send_done);
+  });
+}
+
+TEST(SslTest, OverlappingMatchedTlsCiphersuites) {
+  int port = GetServerPort();
+  const string TLS_CIPHERSUITE_LIST = Substitute("$0:$1", TLS1_3_CIPHERSUITE,
+      TLS1_3_CIPHERSUITE_2);
+  ThriftServer* server;
+  EXPECT_OK(ThriftServerBuilder("DummyStatestore", MakeProcessor(), port)
+      .ssl(SERVER_CERT, PASSWORD_PROTECTED_PRIVATE_KEY)
+      .pem_password_cmd("echo password")
+      .disable_tls12(true)
+      .tls_ciphersuites(TLS_CIPHERSUITE_LIST)
+      .Build(&server));
+  EXPECT_OK(server->Start());
+
+  FLAGS_ssl_client_ca_certificate = SERVER_CERT;
+  auto s = ScopedFlagSetter<string>::Make(&FLAGS_tls_ciphersuites, TLS1_3_CIPHERSUITE);
+  ThriftClient<StatestoreServiceClientWrapper> ssl_client(
+      "localhost", port, "", nullptr, true, true /* disable_tls12 */);
+
   EXPECT_OK(ssl_client.Open());
 
   bool send_done = false;
@@ -462,6 +638,8 @@ TEST(SslTest, OverlappingMatchedCiphers) {
       });
 }
 
+#endif
+
 TEST(ConcurrencyTest, MaxConcurrentConnections) {
   // Tests if max concurrent connections is being enforced by the ThriftServer
   // implementation. It creates a ThriftServer with max_concurrent_connections set to 2
diff --git a/be/src/rpc/thrift-server.cc b/be/src/rpc/thrift-server.cc
index 8dd4dd7b9..dec2671b2 100644
--- a/be/src/rpc/thrift-server.cc
+++ b/be/src/rpc/thrift-server.cc
@@ -38,6 +38,7 @@
 #include "rpc/authentication.h"
 #include "rpc/thrift-server.h"
 #include "rpc/thrift-thread.h"
+#include "rpc/thrift-util.h"
 #include "transport/THttpServer.h"
 #include "util/condition-variable.h"
 #include "util/debug-util.h"
@@ -304,52 +305,10 @@ namespace {
 
 /// Factory subclass to override getPassword() which provides a password string to Thrift
 /// to decrypt the private key file.
-class ImpalaSslSocketFactory : public TSSLSocketFactory {
+class ImpalaPasswordedTlsSocketFactory : public ImpalaTlsSocketFactory {
  public:
-  ImpalaSslSocketFactory(SSLProtocol version, const string& password)
-    : TSSLSocketFactory(version), password_(password) {}
-
-  void ciphers(const string& enable) override {
-    if (ctx_.get() == nullptr) {
-      throw new TSSLException("ImpalaSslSocketFactory was not properly initialized.");
-    }
-    LOG(INFO) << "Enabling the following ciphers for the ImpalaSslSocketFactory: "
-              << enable;
-    SCOPED_OPENSSL_NO_PENDING_ERRORS;
-    TSSLSocketFactory::ciphers(enable);
-
-    // The following was taken from be/src/kudu/security/tls_context.cc, bugs fixed here
-    // may also need to be fixed there.
-    // Enable ECDH curves. For OpenSSL 1.1.0 and up, this is done automatically.
-#ifndef OPENSSL_NO_ECDH
-#if OPENSSL_VERSION_NUMBER < 0x10002000L
-    // OpenSSL 1.0.1 and below only support setting a single ECDH curve at once.
-    // We choose prime256v1 because it's the first curve listed in the "modern
-    // compatibility" section of the Mozilla Server Side TLS recommendations,
-    // accessed Feb. 2017.
-    c_unique_ptr<EC_KEY> ecdh{
-        EC_KEY_new_by_curve_name(NID_X9_62_prime256v1), &EC_KEY_free};
-    if (ecdh == nullptr) {
-      throw TSSLException(
-          "failed to create prime256v1 curve: " + kudu::security::GetOpenSSLErrors());
-    }
-
-    int rc = SSL_CTX_set_tmp_ecdh(ctx_->get(), ecdh.get());
-    if (rc <= 0) {
-      throw new TSSLException(
-          "failed to set ECDH curve: " + kudu::security::GetOpenSSLErrors());
-    }
-#elif OPENSSL_VERSION_NUMBER < 0x10100000L
-    // OpenSSL 1.0.2 provides the set_ecdh_auto API which internally figures out
-    // the best curve to use.
-    int rc = SSL_CTX_set_ecdh_auto(ctx_->get(), 1);
-    if (rc <= 0) {
-      throw TSSLException(
-          "failed to configure ECDH support: " + kudu::security::GetOpenSSLErrors());
-    }
-#endif
-#endif
-  }
+  ImpalaPasswordedTlsSocketFactory(SSLProtocol version, const string& password)
+    : ImpalaTlsSocketFactory(version), password_(password) {}
 
  protected:
   virtual void getPassword(string& output, int size) override {
@@ -372,11 +331,12 @@ Status ThriftServer::CreateSocket(std::shared_ptr<TServerSocket>* socket) {
     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.
-      std::shared_ptr<TSSLSocketFactory> socket_factory(
-          new ImpalaSslSocketFactory(version_, key_password_));
+      std::shared_ptr<ImpalaTlsSocketFactory> socket_factory(
+          new ImpalaPasswordedTlsSocketFactory(version_, key_password_));
       socket_factory->overrideDefaultPasswordCallback();
 
-      if (!cipher_list_.empty()) socket_factory->ciphers(cipher_list_);
+      socket_factory->configureCiphers(cipher_list_, tls_ciphersuites_,
+          disable_tls12_);
       socket_factory->loadCertificate(certificate_path_.c_str());
       socket_factory->loadPrivateKey(private_key_path_.c_str());
       socket->reset(new TSSLServerSocket(port_, socket_factory));
@@ -392,7 +352,8 @@ Status ThriftServer::CreateSocket(std::shared_ptr<TServerSocket>* socket) {
 
 Status ThriftServer::EnableSsl(SSLProtocol version, const string& certificate,
     const string& private_key, const string& pem_password_cmd,
-    const std::string& ciphers) {
+    const std::string& cipher_list, const std::string& tls_ciphersuites,
+    bool disable_tls12) {
   DCHECK(!started_);
   if (certificate.empty()) return Status(TErrorCode::SSL_CERTIFICATE_PATH_BLANK);
   if (private_key.empty()) return Status(TErrorCode::SSL_PRIVATE_KEY_PATH_BLANK);
@@ -409,7 +370,9 @@ Status ThriftServer::EnableSsl(SSLProtocol version, const string& certificate,
   ssl_enabled_ = true;
   certificate_path_ = certificate;
   private_key_path_ = private_key;
-  cipher_list_ = ciphers;
+  cipher_list_ = cipher_list;
+  tls_ciphersuites_ = tls_ciphersuites;
+  disable_tls12_ = disable_tls12;
   version_ = version;
 
   if (!pem_password_cmd.empty()) {
diff --git a/be/src/rpc/thrift-server.h b/be/src/rpc/thrift-server.h
index 0d1113689..7b89fa701 100644
--- a/be/src/rpc/thrift-server.h
+++ b/be/src/rpc/thrift-server.h
@@ -29,6 +29,7 @@
 
 #include "common/status.h"
 #include "gen-cpp/Frontend_types.h"
+#include "kudu/security/security_flags.h"
 #include "util/condition-variable.h"
 #include "util/metrics-fwd.h"
 #include "util/thread.h"
@@ -273,7 +274,10 @@ class ThriftServer {
   /// 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 = "");
+      const std::string& pem_password_cmd = "", const std::string& cipher_list = "",
+      const std::string& tls_ciphersuites =
+          kudu::security::SecurityDefaults::kDefaultTlsCipherSuites,
+      bool disable_tls12 = false);
 
   /// Creates the server socket on which this server listens. May be SSL enabled. Returns
   /// OK unless there was a Thrift error.
@@ -302,6 +306,13 @@ class ThriftServer {
   /// List of ciphers that are ok for clients to use when connecting.
   std::string cipher_list_;
 
+  /// List of TLSv1.3 cipher suites that are ok for clients to use when connecting.
+  std::string tls_ciphersuites_;
+
+  /// Whether to disable TLSv1.2. This is only used for testing TLSv1.3 ciphersuites.
+  /// TODO: Remove this when it is possible to set ssl_minimum_version=TLSv1.3.
+  bool disable_tls12_;
+
   /// The SSL/TLS protocol client versions that this server will allow to connect.
   apache::thrift::transport::SSLProtocol version_;
 
@@ -428,8 +439,22 @@ class ThriftServerBuilder {
 
   /// Sets the list of acceptable cipher suites for this server. Default is to use all
   /// available system cipher suites.
-  ThriftServerBuilder& cipher_list(const std::string& ciphers) {
-    ciphers_ = ciphers;
+  ThriftServerBuilder& cipher_list(const std::string& cipher_list) {
+    cipher_list_ = cipher_list;
+    return *this;
+  }
+
+  /// Sets the list of TLS 1.3 ciphersuites for this server. Default is to
+  /// use all available TLS 1.3 ciphersuites.
+  ThriftServerBuilder& tls_ciphersuites(const std::string& tls_ciphersuites) {
+    tls_ciphersuites_ = tls_ciphersuites;
+    return *this;
+  }
+
+  /// Sets whether to disable TLS 1.2. This is used for testing TLS 1.3.
+  /// TODO: Remove this when ssl_minimum_version=tlsv1.3 is supported.
+  ThriftServerBuilder& disable_tls12(bool disable) {
+    disable_tls12_ = disable;
     return *this;
   }
 
@@ -450,7 +475,8 @@ class ThriftServerBuilder {
             server_transport_type_));
     if (enable_ssl_) {
       RETURN_IF_ERROR(ptr->EnableSsl(
-          version_, certificate_, private_key_, pem_password_cmd_, ciphers_));
+          version_, certificate_, private_key_, pem_password_cmd_, cipher_list_,
+          tls_ciphersuites_, disable_tls12_));
     }
     (*server) = ptr.release();
     return Status::OK();
@@ -475,7 +501,10 @@ class ThriftServerBuilder {
   std::string certificate_;
   std::string private_key_;
   std::string pem_password_cmd_;
-  std::string ciphers_;
+  std::string cipher_list_;
+  std::string tls_ciphersuites_ =
+      kudu::security::SecurityDefaults::kDefaultTlsCipherSuites;
+  bool disable_tls12_ = false;
 };
 
 /// Contains a map from string for --ssl_minimum_version to Thrift's SSLProtocol.
diff --git a/be/src/rpc/thrift-util.cc b/be/src/rpc/thrift-util.cc
index 4f2e31e4f..bef44402d 100644
--- a/be/src/rpc/thrift-util.cc
+++ b/be/src/rpc/thrift-util.cc
@@ -19,7 +19,10 @@
 
 #include <thrift/config.h>
 
+#include "kudu/security/security_flags.h"
+#include "kudu/util/openssl_util.h"
 #include "util/hash-util.h"
+#include "util/openssl-util.h"
 #include "util/time.h"
 #include "rpc/thrift-server.h"
 #include "gen-cpp/Data_types.h"
@@ -95,6 +98,93 @@ std::shared_ptr<TProtocol> CreateDeserializeProtocol(
   }
 }
 
+void ImpalaTlsSocketFactory::configureCiphers(const string& cipher_list,
+    const string& tls_ciphersuites, bool disable_tls12) {
+  if (cipher_list.empty() &&
+      tls_ciphersuites == kudu::security::SecurityDefaults::kDefaultTlsCipherSuites &&
+      !disable_tls12) {
+    return;
+  }
+  if (ctx_.get() == nullptr) {
+    throw TSSLException("ImpalaSslSocketFactory was not properly initialized.");
+  }
+#if OPENSSL_VERSION_NUMBER >= 0x10101000L
+  // Disabling TLS 1.2 only makes sense if OpenSSL supports TLS 1.3.
+  if (disable_tls12) {
+    SCOPED_OPENSSL_NO_PENDING_ERRORS;
+    // This is a setting used for testing TLS 1.3 cipher suites.
+    LOG(INFO) << "TLS 1.2 is disabled.";
+    long options = SSL_CTX_get_options(ctx_->get());
+    options |= SSL_OP_NO_TLSv1_2;
+    SSL_CTX_set_options(ctx_->get(), options);
+  }
+  if (tls_ciphersuites != kudu::security::SecurityDefaults::kDefaultTlsCipherSuites) {
+    SCOPED_OPENSSL_NO_PENDING_ERRORS;
+    if (tls_ciphersuites.empty()) {
+      LOG(INFO) << "TLS 1.3 cipher suites are disabled.";
+      // If there are no TLS 1.3 cipher suites, disable TLS 1.3. Otherwise, the
+      // client/server negotiates TLS 1.3 but then doesn't have any ciphers.
+      long options = SSL_CTX_get_options(ctx_->get());
+      options |= SSL_OP_NO_TLSv1_3;
+      SSL_CTX_set_options(ctx_->get(), options);
+    } else {
+      LOG(INFO) << "Enabling the following TLS 1.3 cipher suites for the "
+                << "ImpalaSslSocketFactory: "
+                << tls_ciphersuites;
+    }
+    int retval = SSL_CTX_set_ciphersuites(ctx_->get(), tls_ciphersuites.c_str());
+    const string& openssl_err = kudu::security::GetOpenSSLErrors();
+    if (retval <= 0 || !openssl_err.empty()) {
+      LOG(INFO) << "SSL_CTX_set_ciphersuites failed: "
+                << openssl_err;
+      throw TSSLException("SSL_CTX_set_ciphersuites: " + openssl_err);
+    }
+  }
+#endif
+
+  SCOPED_OPENSSL_NO_PENDING_ERRORS;
+  if (!cipher_list.empty()) {
+    LOG(INFO) << "Enabling the following TLS 1.2 and below ciphers for the "
+              << "ImpalaSslSocketFactory: "
+              << cipher_list;
+    TSSLSocketFactory::ciphers(cipher_list);
+  }
+
+  // The following was taken from be/src/kudu/security/tls_context.cc, bugs fixed here
+  // may also need to be fixed there.
+  // Enable ECDH curves. For OpenSSL 1.1.0 and up, this is done automatically.
+#ifndef OPENSSL_NO_ECDH
+#if OPENSSL_VERSION_NUMBER < 0x10002000L
+  // TODO: OpenSSL 1.0.1 is old. Centos 7.4 and above use 1.0.2. This probably can
+  // be removed.
+  // OpenSSL 1.0.1 and below only support setting a single ECDH curve at once.
+  // We choose prime256v1 because it's the first curve listed in the "modern
+  // compatibility" section of the Mozilla Server Side TLS recommendations,
+  // accessed Feb. 2017.
+  c_unique_ptr<EC_KEY> ecdh{
+      EC_KEY_new_by_curve_name(NID_X9_62_prime256v1), &EC_KEY_free};
+  if (ecdh == nullptr) {
+    throw TSSLException(
+        "failed to create prime256v1 curve: " + kudu::security::GetOpenSSLErrors());
+  }
+
+  int rc = SSL_CTX_set_tmp_ecdh(ctx_->get(), ecdh.get());
+  if (rc <= 0) {
+    throw TSSLException(
+        "failed to set ECDH curve: " + kudu::security::GetOpenSSLErrors());
+  }
+#elif OPENSSL_VERSION_NUMBER < 0x10100000L
+  // OpenSSL 1.0.2 provides the set_ecdh_auto API which internally figures out
+  // the best curve to use.
+  int rc = SSL_CTX_set_ecdh_auto(ctx_->get(), 1);
+  if (rc <= 0) {
+    throw TSSLException(
+        "failed to configure ECDH support: " + kudu::security::GetOpenSSLErrors());
+  }
+#endif
+#endif
+}
+
 static void ThriftOutputFunction(const char* output) {
   VLOG_QUERY << output;
 }
diff --git a/be/src/rpc/thrift-util.h b/be/src/rpc/thrift-util.h
index 948470f51..99e760e86 100644
--- a/be/src/rpc/thrift-util.h
+++ b/be/src/rpc/thrift-util.h
@@ -26,6 +26,7 @@
 #include <thrift/TApplicationException.h>
 #include <thrift/protocol/TDebugProtocol.h>
 #include <thrift/transport/TBufferTransports.h>
+#include <thrift/transport/TSSLSocket.h>
 #include <thrift/transport/TTransportException.h>
 
 #include "common/status.h"
@@ -126,6 +127,19 @@ Status DeserializeThriftMsg(const uint8_t* buf, uint32_t* len, bool compact,
   return Status::OK();
 }
 
+class ImpalaTlsSocketFactory : public apache::thrift::transport::TSSLSocketFactory {
+ public:
+  ImpalaTlsSocketFactory(apache::thrift::transport::SSLProtocol version)
+    : TSSLSocketFactory(version) {}
+
+  // 'cipher_list': TLS1.2 and below cipher list
+  // 'tls_ciphersuites': TLS1.3 and above cipher suites
+  // 'disable_tls12': Whether to disable TLS1.2 (used for testing TLS1.3).
+  void configureCiphers(const string& cipher_list, const string& tls_ciphersuites,
+      bool disable_tls12);
+};
+
+
 /// Redirects all Thrift logging to VLOG(1)
 void InitThriftLogging();
 
diff --git a/be/src/service/impala-server.cc b/be/src/service/impala-server.cc
index 31582692a..5d7b77dbc 100644
--- a/be/src/service/impala-server.cc
+++ b/be/src/service/impala-server.cc
@@ -243,6 +243,13 @@ DEFINE_string(ssl_cipher_list, SecurityDefaults::kDefaultTlsCiphers,
     "ciphers for more information. If empty, the default cipher list for your platform "
     "is used");
 
+DEFINE_string(tls_ciphersuites,
+    kudu::security::SecurityDefaults::kDefaultTlsCipherSuites,
+    "The TLSv1.3 cipher suites to use for TLS-secured Thrift RPC and KRPC connections. "
+    "TLSv1.3 uses a new way to specify ciper suites that is independent of the older "
+    "TLSv1.2 and below cipher lists. See 'man (1) ciphers' for more information. "
+    "This flag is only effective if Impala is built with OpenSSL v1.1.1 or newer.");
+
 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 "
     "TLSv1.0, TLSv1.1 and TLSv1.2 (as long as the system OpenSSL library supports them)";
diff --git a/be/src/thirdparty/squeasel/squeasel.c b/be/src/thirdparty/squeasel/squeasel.c
index 40659ec80..1ba9aa8bd 100644
--- a/be/src/thirdparty/squeasel/squeasel.c
+++ b/be/src/thirdparty/squeasel/squeasel.c
@@ -168,8 +168,12 @@ static const char *http_500_error = "Internal Server Error";
 #ifndef SSL_OP_NO_TLSv1_1
 #define SSL_OP_NO_TLSv1_1 0x10000000U
 #endif
+#ifndef SSL_OP_NO_TLSv1_2
+#define SSL_OP_NO_TLSv1_2 0x08000000U
+#endif
 
 #define OPENSSL_MIN_VERSION_WITH_TLS_1_1 0x10001000L
+#define OPENSSL_MIN_VERSION_WITH_TLS_1_3 0x10101000L
 
 static const char *month_names[] = {
   "Jan", "Feb", "Mar", "Apr", "May", "Jun",
@@ -230,7 +234,7 @@ enum {
   GLOBAL_PASSWORDS_FILE, INDEX_FILES, ENABLE_KEEP_ALIVE, ACCESS_CONTROL_LIST,
   EXTRA_MIME_TYPES, LISTENING_PORTS, DOCUMENT_ROOT, SSL_CERTIFICATE, SSL_PRIVATE_KEY,
   SSL_PRIVATE_KEY_PASSWORD, SSL_GLOBAL_INIT, NUM_THREADS, RUN_AS_USER, REWRITE,
-  HIDE_FILES, REQUEST_TIMEOUT, SSL_VERSION, SSL_CIPHERS, NUM_OPTIONS
+  HIDE_FILES, REQUEST_TIMEOUT, SSL_VERSION, SSL_CIPHERS, TLS_CIPHERSUITES, NUM_OPTIONS
 };
 
 static const char *config_options[] = {
@@ -264,6 +268,7 @@ static const char *config_options[] = {
   "request_timeout_ms", "30000",
   "ssl_min_version", "tlsv1",
   "ssl_ciphers", NULL,
+  "tls_ciphersuites", NULL,
   NULL
 };
 
@@ -3884,6 +3889,12 @@ static int set_ssl_option(struct sq_context *ctx) {
       return 0;
     }
     options |= (SSL_OP_NO_TLSv1 | SSL_OP_NO_TLSv1_1);
+  } else if (sq_strcasecmp(ssl_version, "tlsv1.3") == 0) {
+    if (SSLeay() < OPENSSL_MIN_VERSION_WITH_TLS_1_3) {
+      cry(fc(ctx), "Unsupported TLS version: %s", ssl_version);
+      return 0;
+    }
+    options |= (SSL_OP_NO_TLSv1 | SSL_OP_NO_TLSv1_1 | SSL_OP_NO_TLSv1_2);
   } else {
     cry(fc(ctx), "%s: unknown SSL version: %s", __func__, ssl_version);
     return 0;
@@ -3944,6 +3955,17 @@ static int set_ssl_option(struct sq_context *ctx) {
     (void) SSL_CTX_use_certificate_chain_file(ctx->ssl_ctx, pem);
   }
 
+  if (ctx->config[TLS_CIPHERSUITES] != NULL) {
+#if OPENSSL_VERSION_NUMBER >= 0x10101000L
+    // Set TLSv1.3 ciphers.
+    if (SSL_CTX_set_ciphersuites(ctx->ssl_ctx, ctx->config[TLS_CIPHERSUITES]) == 0) {
+      cry(fc(ctx), "SSL_CTX_set_ciphersuites: error setting ciphersuites (%s): %s",
+          ctx->config[TLS_CIPHERSUITES], ssl_error());
+      return 0;
+    }
+#endif
+  }
+
   if (ctx->config[SSL_CIPHERS] != NULL) {
     if (SSL_CTX_set_cipher_list(ctx->ssl_ctx, ctx->config[SSL_CIPHERS]) == 0) {
       cry(fc(ctx), "SSL_CTX_set_cipher_list: error setting ciphers (%s): %s",
diff --git a/be/src/util/webserver-test.cc b/be/src/util/webserver-test.cc
index 8b34974c0..65e18f640 100644
--- a/be/src/util/webserver-test.cc
+++ b/be/src/util/webserver-test.cc
@@ -46,6 +46,7 @@ DECLARE_string(webserver_private_key_password_cmd);
 DECLARE_string(webserver_x_frame_options);
 DECLARE_string(ssl_cipher_list);
 DECLARE_string(ssl_minimum_version);
+DECLARE_string(tls_ciphersuites);
 DECLARE_bool(webserver_ldap_passwords_in_clear_ok);
 DECLARE_bool(cookie_require_secure);
 
@@ -280,24 +281,57 @@ TEST(Webserver, SslCipherSuite) {
       Substitute("$0/be/src/testutil/server-key-password.pem", getenv("IMPALA_HOME")));
   auto cmd = ScopedFlagSetter<string>::Make(
       &FLAGS_webserver_private_key_password_cmd, "echo password");
-#ifndef __aarch64__
   {
     auto ciphers = ScopedFlagSetter<string>::Make(
         &FLAGS_ssl_cipher_list, "not_a_cipher");
+    auto ciphersuites = ScopedFlagSetter<string>::Make(
+        &FLAGS_tls_ciphersuites, "");
     MetricGroup metrics("webserver-test");
     Webserver webserver("", FLAGS_webserver_port, &metrics);
     ASSERT_FALSE(webserver.Start().ok());
   }
-#endif
   {
     auto ciphers = ScopedFlagSetter<string>::Make(
         &FLAGS_ssl_cipher_list, "AES128-SHA");
+    auto ciphersuites = ScopedFlagSetter<string>::Make(
+        &FLAGS_tls_ciphersuites, "");
+    MetricGroup metrics("webserver-test");
+    Webserver webserver("", FLAGS_webserver_port, &metrics);
+    ASSERT_OK(webserver.Start());
+  }
+}
+
+#if OPENSSL_VERSION_NUMBER >= 0x10101000L
+
+TEST(Webserver, TlsCiphersuite) {
+  auto cert = ScopedFlagSetter<string>::Make(&FLAGS_webserver_certificate_file,
+      Substitute("$0/be/src/testutil/server-cert.pem", getenv("IMPALA_HOME")));
+  auto key = ScopedFlagSetter<string>::Make(&FLAGS_webserver_private_key_file,
+      Substitute("$0/be/src/testutil/server-key-password.pem", getenv("IMPALA_HOME")));
+  auto cmd = ScopedFlagSetter<string>::Make(
+      &FLAGS_webserver_private_key_password_cmd, "echo password");
+  {
+    auto ciphers = ScopedFlagSetter<string>::Make(
+        &FLAGS_ssl_minimum_version, "tlsv1.3");
+    auto ciphersuites = ScopedFlagSetter<string>::Make(
+        &FLAGS_tls_ciphersuites, "not_a_ciphersuite");
+    MetricGroup metrics("webserver-test");
+    Webserver webserver("", FLAGS_webserver_port, &metrics);
+    ASSERT_FALSE(webserver.Start().ok());
+  }
+  {
+    auto ciphers = ScopedFlagSetter<string>::Make(
+        &FLAGS_ssl_minimum_version, "tlsv1.3");
+    auto ciphersuites = ScopedFlagSetter<string>::Make(
+        &FLAGS_tls_ciphersuites, "TLS_AES_256_GCM_SHA384");
     MetricGroup metrics("webserver-test");
     Webserver webserver("", FLAGS_webserver_port, &metrics);
     ASSERT_OK(webserver.Start());
   }
 }
 
+#endif // OPENSSL_VERSION_NUMBER
+
 TEST(Webserver, SslBadTlsVersion) {
   auto cert = ScopedFlagSetter<string>::Make(&FLAGS_webserver_certificate_file,
       Substitute("$0/be/src/testutil/server-cert.pem", getenv("IMPALA_HOME")));
@@ -321,8 +355,10 @@ TEST(Webserver, SslGoodTlsVersion) {
       Substitute("$0/be/src/testutil/server-key-password.pem", getenv("IMPALA_HOME")));
   auto cmd = ScopedFlagSetter<string>::Make(
       &FLAGS_webserver_private_key_password_cmd, "echo password");
-
-#if OPENSSL_VERSION_NUMBER >= 0x10001000L
+#if OPENSSL_VERSION_NUMBER >= 0x10101000L
+  auto versions = {"tlsv1", "tlsv1.1", "tlsv1.2", "tlsv1.3"};
+  vector <string> unsupported_versions = {};
+#elif OPENSSL_VERSION_NUMBER >= 0x10001000L
   auto versions = {"tlsv1", "tlsv1.1", "tlsv1.2"};
   vector<string> unsupported_versions = {};
 #else
diff --git a/be/src/util/webserver.cc b/be/src/util/webserver.cc
index 0ba67e036..d703ed30e 100644
--- a/be/src/util/webserver.cc
+++ b/be/src/util/webserver.cc
@@ -153,6 +153,7 @@ DECLARE_bool(is_coordinator);
 DECLARE_int64(max_cookie_lifetime_s);
 DECLARE_string(ssl_minimum_version);
 DECLARE_string(ssl_cipher_list);
+DECLARE_string(tls_ciphersuites);
 DECLARE_string(trusted_domain);
 DECLARE_bool(trusted_domain_use_xff_header);
 DECLARE_bool(jwt_token_auth);
@@ -432,6 +433,8 @@ Status Webserver::Start() {
       options.push_back("ssl_ciphers");
       options.push_back(FLAGS_ssl_cipher_list.c_str());
     }
+    options.push_back("tls_ciphersuites");
+    options.push_back(FLAGS_tls_ciphersuites.c_str());
   }
 
   if (!FLAGS_webserver_authentication_domain.empty()) {