You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by da...@apache.org on 2017/02/10 20:42:52 UTC

[2/2] kudu git commit: [rpc] Assume existence of TlsContext during negotiation

[rpc] Assume existence of TlsContext during negotiation

This changes the client and server RPC negotiatiators to always assume
the existence of a TlsContext for the messenger, and check whether there
are valid certs in the context in order to decide whether to use TLS
encryption. This will simplify authentication negotiation down the line.

Change-Id: I806ca360bcd168b1ab075a4c7143436a7ccbba06
Reviewed-on: http://gerrit.cloudera.org:8080/5946
Tested-by: Kudu Jenkins
Reviewed-by: Todd Lipcon <to...@apache.org>
Reviewed-by: Alexey Serbin <as...@cloudera.com>


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

Branch: refs/heads/master
Commit: 60aedaaf4b20a725d7ca2b0f774f73adc5992427
Parents: 18a2ea5
Author: Dan Burkert <da...@apache.org>
Authored: Wed Feb 8 13:32:02 2017 -0800
Committer: Dan Burkert <da...@apache.org>
Committed: Fri Feb 10 20:42:35 2017 +0000

----------------------------------------------------------------------
 src/kudu/rpc/client_negotiation.cc | 21 +++++++------
 src/kudu/rpc/client_negotiation.h  |  9 +++---
 src/kudu/rpc/messenger.cc          | 12 ++++----
 src/kudu/rpc/messenger.h           |  4 ---
 src/kudu/rpc/negotiation-test.cc   | 52 +++++++++++++++++++++------------
 src/kudu/rpc/negotiation.cc        | 11 ++++---
 src/kudu/rpc/server_negotiation.cc | 22 +++++++-------
 src/kudu/rpc/server_negotiation.h  |  9 +++---
 8 files changed, 74 insertions(+), 66 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/60aedaaf/src/kudu/rpc/client_negotiation.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/client_negotiation.cc b/src/kudu/rpc/client_negotiation.cc
index bb1bfb2..6d463b0 100644
--- a/src/kudu/rpc/client_negotiation.cc
+++ b/src/kudu/rpc/client_negotiation.cc
@@ -100,10 +100,11 @@ static Status StatusFromRpcError(const ErrorStatusPB& error) {
   }
 }
 
-ClientNegotiation::ClientNegotiation(unique_ptr<Socket> socket)
+ClientNegotiation::ClientNegotiation(unique_ptr<Socket> socket,
+                                     const security::TlsContext* tls_context)
     : socket_(std::move(socket)),
       helper_(SaslHelper::CLIENT),
-      tls_context_(nullptr),
+      tls_context_(tls_context),
       tls_negotiated_(false),
       negotiated_mech_(SaslMechanism::INVALID),
       deadline_(MonoTime::Max()) {
@@ -114,6 +115,8 @@ ClientNegotiation::ClientNegotiation(unique_ptr<Socket> socket)
   callbacks_.push_back(SaslBuildCallback(SASL_CB_PASS,
       reinterpret_cast<int (*)()>(&ClientNegotiationSecretCb), this));
   callbacks_.push_back(SaslBuildCallback(SASL_CB_LIST_END, nullptr, nullptr));
+  DCHECK(socket_);
+  DCHECK(tls_context_);
 }
 
 Status ClientNegotiation::EnablePlain(const string& user, const string& pass) {
@@ -143,10 +146,6 @@ void ClientNegotiation::set_server_fqdn(const string& domain_name) {
   helper_.set_server_fqdn(domain_name);
 }
 
-void ClientNegotiation::EnableTls(const security::TlsContext* tls_context) {
-  tls_context_ = DCHECK_NOTNULL(tls_context);
-}
-
 void ClientNegotiation::set_deadline(const MonoTime& deadline) {
   deadline_ = deadline;
 }
@@ -170,8 +169,8 @@ Status ClientNegotiation::Negotiate() {
   }
 
   // Step 3: if both ends support TLS, do a TLS handshake.
-  // TODO(dan): allow the client to require TLS.
-  if (tls_context_ && ContainsKey(server_features_, TLS)) {
+  // TODO(PKI): allow the client to require TLS.
+  if (ContainsKey(server_features_, TLS)) {
     RETURN_NOT_OK(tls_context_->InitiateHandshake(security::TlsHandshakeType::CLIENT,
                                                   &tls_handshake_));
 
@@ -179,7 +178,6 @@ Status ClientNegotiation::Negotiate() {
       // When using GSSAPI, we don't verify the server's certificate. Instead,
       // we rely on Kerberos authentication, and use channel binding to tie the
       // SASL authentication to the TLS channel.
-      // TODO(PKI): implement channel binding when TLS and GSSAPI are used.
       tls_handshake_.set_verification_mode(security::TlsVerificationMode::VERIFY_NONE);
     }
 
@@ -351,7 +349,7 @@ Status ClientNegotiation::HandleNegotiate(const NegotiatePB& response) {
                                    "but client does not have Kerberos enabled");
     }
     if (!ContainsKey(server_mechs, SaslMechanism::GSSAPI) &&
-               ContainsKey(client_mechs, SaslMechanism::GSSAPI)) {
+        ContainsKey(client_mechs, SaslMechanism::GSSAPI)) {
       return Status::NotAuthorized("client requires authentication, "
                                    "but server does not have Kerberos enabled");
     }
@@ -367,6 +365,7 @@ Status ClientNegotiation::HandleNegotiate(const NegotiatePB& response) {
     return Status::NotAuthorized(msg);
   }
 
+  // TODO(PKI): allow the client to require authentication.
   if (ContainsKey(common_mechs, SaslMechanism::GSSAPI)) {
     negotiated_mech_ = SaslMechanism::GSSAPI;
   } else {
@@ -505,7 +504,7 @@ Status ClientNegotiation::HandleSaslSuccess(const NegotiatePB& response) {
 
     string expected_channel_bindings;
     RETURN_NOT_OK_PREPEND(cert.GetServerEndPointChannelBindings(&expected_channel_bindings),
-                          "failed to generate expected channel bindings");
+                          "failed to generate channel bindings");
 
     if (!response.has_channel_bindings()) {
       return Status::NotAuthorized("no channel bindings provided by server");

http://git-wip-us.apache.org/repos/asf/kudu/blob/60aedaaf/src/kudu/rpc/client_negotiation.h
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/client_negotiation.h b/src/kudu/rpc/client_negotiation.h
index 0e14ee9..1050302 100644
--- a/src/kudu/rpc/client_negotiation.h
+++ b/src/kudu/rpc/client_negotiation.h
@@ -53,7 +53,10 @@ class ClientNegotiation {
   // provided socket. After completing the negotiation process by setting the
   // desired options and calling Negotiate(), the socket can be retrieved with
   // 'release_socket'.
-  explicit ClientNegotiation(std::unique_ptr<Socket> socket);
+  //
+  // The provided TlsContext must outlive this negotiation instance.
+  explicit ClientNegotiation(std::unique_ptr<Socket> socket,
+                             const security::TlsContext* tls_context);
 
   // Enable PLAIN authentication.
   // Must be called before Negotiate().
@@ -93,10 +96,6 @@ class ClientNegotiation {
   // Must be called before Negotiate(). Required for some mechanisms.
   void set_server_fqdn(const std::string& domain_name);
 
-  // Allow TLS to be used on the connection. 'tls_context' must outlive this
-  // ClientNegotiation.
-  void EnableTls(const security::TlsContext* tls_context);
-
   // Set deadline for connection negotiation.
   void set_deadline(const MonoTime& deadline);
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/60aedaaf/src/kudu/rpc/messenger.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/messenger.cc b/src/kudu/rpc/messenger.cc
index da2f364..2dd21fe 100644
--- a/src/kudu/rpc/messenger.cc
+++ b/src/kudu/rpc/messenger.cc
@@ -270,6 +270,7 @@ void Messenger::RegisterInboundSocket(Socket *new_socket, const Sockaddr &remote
 Messenger::Messenger(const MessengerBuilder &bld)
   : name_(bld.name_),
     closing_(false),
+    tls_context_(new security::TlsContext()),
     rpcz_store_(new RpczStore()),
     metric_entity_(bld.metric_entity_),
     retain_self_(this) {
@@ -298,17 +299,14 @@ Reactor* Messenger::RemoteToReactor(const Sockaddr &remote) {
 
 Status Messenger::Init() {
   Status status;
-  server_tls_enabled_ = !FLAGS_rpc_ssl_server_certificate.empty()
-                     || !FLAGS_rpc_ssl_private_key.empty()
-                     || !FLAGS_rpc_ssl_certificate_authority.empty();
 
-  // Enable TLS unconditionally for client connections.
-  tls_context_.reset(new security::TlsContext());
   RETURN_NOT_OK(tls_context_->Init());
-  if (server_tls_enabled_) {
+  if (!FLAGS_rpc_ssl_server_certificate.empty() ||
+      !FLAGS_rpc_ssl_private_key.empty() ||
+      !FLAGS_rpc_ssl_certificate_authority.empty()) {
+    RETURN_NOT_OK(tls_context_->LoadCertificateAuthority(FLAGS_rpc_ssl_certificate_authority));
     RETURN_NOT_OK(tls_context_->LoadCertificateAndKey(FLAGS_rpc_ssl_server_certificate,
                                                       FLAGS_rpc_ssl_private_key));
-    RETURN_NOT_OK(tls_context_->LoadCertificateAuthority(FLAGS_rpc_ssl_certificate_authority));
   }
   for (Reactor* r : reactors_) {
     RETURN_NOT_OK(r->Init());

http://git-wip-us.apache.org/repos/asf/kudu/blob/60aedaaf/src/kudu/rpc/messenger.h
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/messenger.h b/src/kudu/rpc/messenger.h
index 0bf4d07..8303361 100644
--- a/src/kudu/rpc/messenger.h
+++ b/src/kudu/rpc/messenger.h
@@ -198,8 +198,6 @@ class Messenger {
     return name_;
   }
 
-  bool server_tls_enabled() const { return server_tls_enabled_; }
-
   bool closing() const {
     shared_lock<rw_spinlock> l(lock_.get_lock());
     return closing_;
@@ -230,8 +228,6 @@ class Messenger {
 
   bool closing_;
 
-  bool server_tls_enabled_;
-
   // Pools which are listening on behalf of this messenger.
   // Note that the user may have called Shutdown() on one of these
   // pools, so even though we retain the reference, it may no longer

http://git-wip-us.apache.org/repos/asf/kudu/blob/60aedaaf/src/kudu/rpc/negotiation-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/negotiation-test.cc b/src/kudu/rpc/negotiation-test.cc
index 7efbe31..37aff51 100644
--- a/src/kudu/rpc/negotiation-test.cc
+++ b/src/kudu/rpc/negotiation-test.cc
@@ -111,7 +111,9 @@ static void RunNegotiationTest(const SocketCallable& server_runner,
 ////////////////////////////////////////////////////////////////////////////////
 
 static void RunPlainNegotiationServer(unique_ptr<Socket> socket) {
-  ServerNegotiation server_negotiation(std::move(socket));
+  security::TlsContext tls_context;
+  CHECK_OK(tls_context.Init());
+  ServerNegotiation server_negotiation(std::move(socket), &tls_context);
   CHECK_OK(server_negotiation.EnablePlain());
   CHECK_OK(server_negotiation.Negotiate());
   CHECK(ContainsKey(server_negotiation.client_features(), APPLICATION_FEATURE_FLAGS));
@@ -119,7 +121,9 @@ static void RunPlainNegotiationServer(unique_ptr<Socket> socket) {
 }
 
 static void RunPlainNegotiationClient(unique_ptr<Socket> socket) {
-  ClientNegotiation client_negotiation(std::move(socket));
+  security::TlsContext tls_context;
+  CHECK_OK(tls_context.Init());
+  ClientNegotiation client_negotiation(std::move(socket), &tls_context);
   CHECK_OK(client_negotiation.EnablePlain("my-username", "ignored password"));
   CHECK_OK(client_negotiation.Negotiate());
   CHECK(ContainsKey(client_negotiation.server_features(), APPLICATION_FEATURE_FLAGS));
@@ -140,7 +144,9 @@ using CheckerFunction = std::function<void(const Status&, T&)>;
 // 'post_check' after negotiation to verify the result.
 static void RunGSSAPINegotiationServer(unique_ptr<Socket> socket,
                                        const CheckerFunction<ServerNegotiation>& post_check) {
-  ServerNegotiation server_negotiation(std::move(socket));
+  security::TlsContext tls_context;
+  CHECK_OK(tls_context.Init());
+  ServerNegotiation server_negotiation(std::move(socket), &tls_context);
   server_negotiation.set_server_fqdn("127.0.0.1");
   CHECK_OK(server_negotiation.EnableGSSAPI());
   post_check(server_negotiation.Negotiate(), server_negotiation);
@@ -150,7 +156,9 @@ static void RunGSSAPINegotiationServer(unique_ptr<Socket> socket,
 // 'post_check' after negotiation to verify the result.
 static void RunGSSAPINegotiationClient(unique_ptr<Socket> conn,
                                        const CheckerFunction<ClientNegotiation>& post_check) {
-  ClientNegotiation client_negotiation(std::move(conn));
+  security::TlsContext tls_context;
+  CHECK_OK(tls_context.Init());
+  ClientNegotiation client_negotiation(std::move(conn), &tls_context);
   client_negotiation.set_server_fqdn("127.0.0.1");
   CHECK_OK(client_negotiation.EnableGSSAPI());
   post_check(client_negotiation.Negotiate(), client_negotiation);
@@ -183,7 +191,9 @@ TEST_F(TestNegotiation, TestRestrictiveServer_NonRestrictiveClient) {
                   CHECK_EQ("testuser", server.authenticated_user());
                 }),
       [](unique_ptr<Socket> socket) {
-        ClientNegotiation client_negotiation(std::move(socket));
+        security::TlsContext tls_context;
+        CHECK_OK(tls_context.Init());
+        ClientNegotiation client_negotiation(std::move(socket), &tls_context);
         client_negotiation.set_server_fqdn("127.0.0.1");
         // The client enables both PLAIN and GSSAPI.
         CHECK_OK(client_negotiation.EnablePlain("foo", "bar"));
@@ -216,7 +226,9 @@ TEST_F(TestNegotiation, TestNoMatchingMechanisms) {
                   ASSERT_STR_CONTAINS(s.ToString(), "got EOF from remote");
                 }),
       [](unique_ptr<Socket> socket) {
-        ClientNegotiation client_negotiation(std::move(socket));
+        security::TlsContext tls_context;
+        CHECK_OK(tls_context.Init());
+        ClientNegotiation client_negotiation(std::move(socket), &tls_context);
         client_negotiation.set_server_fqdn("127.0.0.1");
         // The client enables both PLAIN and GSSAPI.
         CHECK_OK(client_negotiation.EnablePlain("foo", "bar"));
@@ -387,7 +399,9 @@ TEST_F(TestNegotiation, TestPreflight) {
 ////////////////////////////////////////////////////////////////////////////////
 
 static void RunTimeoutExpectingServer(unique_ptr<Socket> socket) {
-  ServerNegotiation server_negotiation(std::move(socket));
+  security::TlsContext tls_context;
+  CHECK_OK(tls_context.Init());
+  ServerNegotiation server_negotiation(std::move(socket), &tls_context);
   CHECK_OK(server_negotiation.EnablePlain());
   Status s = server_negotiation.Negotiate();
   ASSERT_TRUE(s.IsNetworkError()) << "Expected client to time out and close the connection. Got: "
@@ -395,7 +409,9 @@ static void RunTimeoutExpectingServer(unique_ptr<Socket> socket) {
 }
 
 static void RunTimeoutNegotiationClient(unique_ptr<Socket> sock) {
-  ClientNegotiation client_negotiation(std::move(sock));
+  security::TlsContext tls_context;
+  CHECK_OK(tls_context.Init());
+  ClientNegotiation client_negotiation(std::move(sock), &tls_context);
   CHECK_OK(client_negotiation.EnablePlain("test", "test"));
   MonoTime deadline = MonoTime::Now() - MonoDelta::FromMilliseconds(100L);
   client_negotiation.set_deadline(deadline);
@@ -412,7 +428,9 @@ TEST_F(TestNegotiation, TestClientTimeout) {
 ////////////////////////////////////////////////////////////////////////////////
 
 static void RunTimeoutNegotiationServer(unique_ptr<Socket> socket) {
-  ServerNegotiation server_negotiation(std::move(socket));
+  security::TlsContext tls_context;
+  CHECK_OK(tls_context.Init());
+  ServerNegotiation server_negotiation(std::move(socket), &tls_context);
   CHECK_OK(server_negotiation.EnablePlain());
   MonoTime deadline = MonoTime::Now() - MonoDelta::FromMilliseconds(100L);
   server_negotiation.set_deadline(deadline);
@@ -422,7 +440,9 @@ static void RunTimeoutNegotiationServer(unique_ptr<Socket> socket) {
 }
 
 static void RunTimeoutExpectingClient(unique_ptr<Socket> socket) {
-  ClientNegotiation client_negotiation(std::move(socket));
+  security::TlsContext tls_context;
+  CHECK_OK(tls_context.Init());
+  ClientNegotiation client_negotiation(std::move(socket), &tls_context);
   CHECK_OK(client_negotiation.EnablePlain("test", "test"));
   Status s = client_negotiation.Negotiate();
   ASSERT_TRUE(s.IsNetworkError()) << "Expected server to time out and close the connection. Got: "
@@ -542,19 +562,17 @@ TEST_F(TestDisableInit, TestMultipleSaslInit_NoMutexImpl) {
 
 // Server which has TLS and SASL GSSAPI enabled.
 static void RunTlsGssapiNegotiationServer(unique_ptr<Socket> socket) {
-  ServerNegotiation server_negotiation(std::move(socket));
+  security::TlsContext tls_context;
+  ASSERT_OK(tls_context.Init());
 
   // TODO(PKI): switch this to use an in-memory cert and key.
   std::string server_cert_path = JoinPathSegments(GetTestDataDirectory(), "server-cert.pem");
   std::string private_key_path = JoinPathSegments(GetTestDataDirectory(), "server-key.pem");
   ASSERT_OK(security::CreateSSLServerCert(server_cert_path));
   ASSERT_OK(security::CreateSSLPrivateKey(private_key_path));
-
-  security::TlsContext tls_context;
-  ASSERT_OK(tls_context.Init());
   ASSERT_OK(tls_context.LoadCertificateAndKey(server_cert_path, private_key_path));
-  server_negotiation.EnableTls(&tls_context);
 
+  ServerNegotiation server_negotiation(std::move(socket), &tls_context);
   server_negotiation.set_server_fqdn("127.0.0.1");
   ASSERT_OK(server_negotiation.EnableGSSAPI());
 
@@ -569,12 +587,10 @@ static void RunTlsGssapiNegotiationServer(unique_ptr<Socket> socket) {
 }
 
 static void RunTlsGssapiNegotiationClient(unique_ptr<Socket> socket) {
-  ClientNegotiation client_negotiation(std::move(socket));
-
   security::TlsContext tls_context;
   ASSERT_OK(tls_context.Init());
-  client_negotiation.EnableTls(&tls_context);
 
+  ClientNegotiation client_negotiation(std::move(socket), &tls_context);
   client_negotiation.set_server_fqdn("127.0.0.1");
   CHECK_OK(client_negotiation.EnableGSSAPI());
   CHECK_OK(client_negotiation.Negotiate());

http://git-wip-us.apache.org/repos/asf/kudu/blob/60aedaaf/src/kudu/rpc/negotiation.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/negotiation.cc b/src/kudu/rpc/negotiation.cc
index 0f490a2..4e86292 100644
--- a/src/kudu/rpc/negotiation.cc
+++ b/src/kudu/rpc/negotiation.cc
@@ -129,7 +129,8 @@ static Status DisableSocketTimeouts(Socket* socket) {
 
 // Perform client negotiation. We don't LOG() anything, we leave that to our caller.
 static Status DoClientNegotiation(Connection* conn, MonoTime deadline) {
-  ClientNegotiation client_negotiation(conn->release_socket());
+  const auto* tls_context = &conn->reactor_thread()->reactor()->messenger()->tls_context();
+  ClientNegotiation client_negotiation(conn->release_socket(), tls_context);
 
   // Note that the fqdn is an IP address here: we've already lost whatever DNS
   // name the client was attempting to use. Unless krb5 is configured with 'rdns
@@ -154,7 +155,6 @@ static Status DoClientNegotiation(Connection* conn, MonoTime deadline) {
   }
 
   RETURN_NOT_OK(client_negotiation.EnablePlain(conn->user_credentials().real_user(), ""));
-  client_negotiation.EnableTls(&conn->reactor_thread()->reactor()->messenger()->tls_context());
   client_negotiation.set_deadline(deadline);
 
   RETURN_NOT_OK(WaitForClientConnect(client_negotiation.socket(), deadline));
@@ -178,15 +178,14 @@ static Status DoServerNegotiation(Connection* conn, const MonoTime& deadline) {
   }
 
   // Create a new ServerNegotiation to handle the synchronous negotiation.
-  ServerNegotiation server_negotiation(conn->release_socket());
+  const auto* tls_context = &conn->reactor_thread()->reactor()->messenger()->tls_context();
+  ServerNegotiation server_negotiation(conn->release_socket(), tls_context);
+
   if (FLAGS_server_require_kerberos) {
     RETURN_NOT_OK(server_negotiation.EnableGSSAPI());
   } else {
     RETURN_NOT_OK(server_negotiation.EnablePlain());
   }
-  if (conn->reactor_thread()->reactor()->messenger()->server_tls_enabled()) {
-    server_negotiation.EnableTls(&conn->reactor_thread()->reactor()->messenger()->tls_context());
-  }
   server_negotiation.set_deadline(deadline);
 
   RETURN_NOT_OK(server_negotiation.socket()->SetNonBlocking(false));

http://git-wip-us.apache.org/repos/asf/kudu/blob/60aedaaf/src/kudu/rpc/server_negotiation.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/server_negotiation.cc b/src/kudu/rpc/server_negotiation.cc
index d52eb75..fc5bcdc 100644
--- a/src/kudu/rpc/server_negotiation.cc
+++ b/src/kudu/rpc/server_negotiation.cc
@@ -67,10 +67,11 @@ static int ServerNegotiationPlainAuthCb(sasl_conn_t* conn,
   return server_negotiation->PlainAuthCb(conn, user, pass, passlen, propctx);
 }
 
-ServerNegotiation::ServerNegotiation(unique_ptr<Socket> socket)
+ServerNegotiation::ServerNegotiation(unique_ptr<Socket> socket,
+                                     const security::TlsContext* tls_context)
     : socket_(std::move(socket)),
       helper_(SaslHelper::SERVER),
-      tls_context_(nullptr),
+      tls_context_(tls_context),
       tls_negotiated_(false),
       negotiated_mech_(SaslMechanism::INVALID),
       deadline_(MonoTime::Max()) {
@@ -110,10 +111,6 @@ void ServerNegotiation::set_server_fqdn(const string& domain_name) {
   helper_.set_server_fqdn(domain_name);
 }
 
-void ServerNegotiation::EnableTls(const security::TlsContext* tls_context) {
-  tls_context_ = DCHECK_NOTNULL(tls_context);
-}
-
 void ServerNegotiation::set_deadline(const MonoTime& deadline) {
   deadline_ = deadline;
 }
@@ -121,6 +118,12 @@ void ServerNegotiation::set_deadline(const MonoTime& deadline) {
 Status ServerNegotiation::Negotiate() {
   TRACE("Beginning negotiation");
 
+  // Wait until starting negotiation to check that the socket and tls_context
+  // are not null, since the socket and tls_context need not be set for
+  // PreflightCheckGSSAPI.
+  DCHECK(socket_);
+  DCHECK(tls_context_);
+
   // Ensure we can use blocking calls on the socket during negotiation.
   RETURN_NOT_OK(EnsureBlockingMode(socket_.get()));
 
@@ -137,7 +140,7 @@ Status ServerNegotiation::Negotiate() {
 
   // Step 3: if both ends support TLS, do a TLS handshake.
   // TODO(dan): allow the server to require TLS.
-  if (tls_context_ && ContainsKey(client_features_, TLS)) {
+  if (tls_context_->has_cert() && ContainsKey(client_features_, TLS)) {
     RETURN_NOT_OK(tls_context_->InitiateHandshake(security::TlsHandshakeType::SERVER,
                                                   &tls_handshake_));
 
@@ -195,7 +198,7 @@ Status ServerNegotiation::PreflightCheckGSSAPI() {
   //
   // We aren't going to actually send/receive any messages, but
   // this makes it easier to reuse the initialization code.
-  ServerNegotiation server(nullptr);
+  ServerNegotiation server(nullptr, nullptr);
   Status s = server.EnableGSSAPI();
   if (!s.ok()) {
     return Status::RuntimeError(s.message());
@@ -364,8 +367,7 @@ Status ServerNegotiation::SendNegotiate(const set<SaslMechanism::Type>& server_m
   for (RpcFeatureFlag feature : kSupportedServerRpcFeatureFlags) {
     response.add_supported_features(feature);
   }
-
-  if (tls_context_) {
+  if (tls_context_->has_cert()) {
     response.add_supported_features(TLS);
   }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/60aedaaf/src/kudu/rpc/server_negotiation.h
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/server_negotiation.h b/src/kudu/rpc/server_negotiation.h
index 88c24d9..682f00b 100644
--- a/src/kudu/rpc/server_negotiation.h
+++ b/src/kudu/rpc/server_negotiation.h
@@ -51,7 +51,10 @@ class ServerNegotiation {
   // provided socket. After completing the negotiation process by setting the
   // desired options and calling Negotiate((), the socket can be retrieved with
   // release_socket().
-  explicit ServerNegotiation(std::unique_ptr<Socket> socket);
+  //
+  // The provided TlsContext must outlive this negotiation instance.
+  explicit ServerNegotiation(std::unique_ptr<Socket> socket,
+                             const security::TlsContext* tls_context);
 
   // Enable PLAIN authentication.
   // Despite PLAIN authentication taking a username and password, we disregard
@@ -96,10 +99,6 @@ class ServerNegotiation {
   // Must be called before Negotiate(). Required for some mechanisms.
   void set_server_fqdn(const std::string& domain_name);
 
-  // Allow TLS to be used on the connection. 'tls_context' must outlive this
-  // ServerNegotiation.
-  void EnableTls(const security::TlsContext* tls_context);
-
   // Set deadline for connection negotiation.
   void set_deadline(const MonoTime& deadline);