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/10/06 21:32:48 UTC

kudu git commit: Allow the SASL protocol service name to be configurable

Repository: kudu
Updated Branches:
  refs/heads/master 858bf73b3 -> 31d58522b


Allow the SASL protocol service name to be configurable

Previously the SASL service name was always set to a constant "kudu"
which was tracked by kSaslProtoName in rpc/constants.h.

However, for applications that use the KRPC library that would prefer
to do their own SASL initialization, they would requre to set their own
SASL service name to be passed into sasl_server_new()/sasl_client_new().

This patch allows for this configuration by adding a configurable
parameter to the MessengerBuilder which is ultimately passed down to the
negotiation layer.

Change-Id: I9e30fe4461893b67527333259579e2304b19af1e
Reviewed-on: http://gerrit.cloudera.org:8080/8218
Reviewed-by: Dan Burkert <da...@apache.org>
Tested-by: Dan Burkert <da...@apache.org>


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

Branch: refs/heads/master
Commit: 31d58522b50aea948f977c7cbc8b64f1b849f323
Parents: 858bf73
Author: Sailesh Mukil <sa...@apache.org>
Authored: Thu Oct 5 09:46:24 2017 -0700
Committer: Dan Burkert <da...@apache.org>
Committed: Fri Oct 6 21:32:38 2017 +0000

----------------------------------------------------------------------
 src/kudu/rpc/client_negotiation.cc |  6 ++++--
 src/kudu/rpc/client_negotiation.h  |  6 +++++-
 src/kudu/rpc/constants.cc          |  1 -
 src/kudu/rpc/constants.h           |  3 ---
 src/kudu/rpc/messenger.cc          |  9 ++++++++-
 src/kudu/rpc/messenger.h           | 11 +++++++++++
 src/kudu/rpc/negotiation-test.cc   | 26 ++++++++++++++------------
 src/kudu/rpc/negotiation.cc        |  6 ++++--
 src/kudu/rpc/server_negotiation.cc | 11 +++++++----
 src/kudu/rpc/server_negotiation.h  |  8 ++++++--
 10 files changed, 59 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/31d58522/src/kudu/rpc/client_negotiation.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/client_negotiation.cc b/src/kudu/rpc/client_negotiation.cc
index de55f29..de69c62 100644
--- a/src/kudu/rpc/client_negotiation.cc
+++ b/src/kudu/rpc/client_negotiation.cc
@@ -107,7 +107,8 @@ static Status StatusFromRpcError(const ErrorStatusPB& error) {
 ClientNegotiation::ClientNegotiation(unique_ptr<Socket> socket,
                                      const security::TlsContext* tls_context,
                                      boost::optional<security::SignedTokenPB> authn_token,
-                                     RpcEncryption encryption)
+                                     RpcEncryption encryption,
+                                     std::string sasl_proto_name)
     : socket_(std::move(socket)),
       helper_(SaslHelper::CLIENT),
       tls_context_(tls_context),
@@ -117,6 +118,7 @@ ClientNegotiation::ClientNegotiation(unique_ptr<Socket> socket,
       psecret_(nullptr, std::free),
       negotiated_authn_(AuthenticationType::INVALID),
       negotiated_mech_(SaslMechanism::INVALID),
+      sasl_proto_name_(std::move(sasl_proto_name)),
       deadline_(MonoTime::Max()) {
   callbacks_.push_back(SaslBuildCallback(SASL_CB_GETOPT,
       reinterpret_cast<int (*)()>(&ClientNegotiationGetoptCb), this));
@@ -284,7 +286,7 @@ Status ClientNegotiation::InitSaslClient() {
   sasl_conn_t* sasl_conn = nullptr;
   RETURN_NOT_OK_PREPEND(WrapSaslCall(nullptr /* no conn */, [&]() {
       return sasl_client_new(
-          kSaslProtoName,               // Registered name of the service using SASL. Required.
+          sasl_proto_name_.c_str(),     // Registered name of the service using SASL. Required.
           helper_.server_fqdn(),        // The fully qualified domain name of the remote server.
           nullptr,                      // Local and remote IP address strings. (we don't use
           nullptr,                      // any mechanisms which require this info.)

http://git-wip-us.apache.org/repos/asf/kudu/blob/31d58522/src/kudu/rpc/client_negotiation.h
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/client_negotiation.h b/src/kudu/rpc/client_negotiation.h
index 37d1136..541798f 100644
--- a/src/kudu/rpc/client_negotiation.h
+++ b/src/kudu/rpc/client_negotiation.h
@@ -64,7 +64,8 @@ class ClientNegotiation {
   ClientNegotiation(std::unique_ptr<Socket> socket,
                     const security::TlsContext* tls_context,
                     boost::optional<security::SignedTokenPB> authn_token,
-                    RpcEncryption encryption);
+                    RpcEncryption encryption,
+                    std::string sasl_proto_name);
 
   // Enable PLAIN authentication.
   // Must be called before Negotiate().
@@ -247,6 +248,9 @@ class ClientNegotiation {
   // The SASL mechanism used by the connection. Filled in during negotiation.
   SaslMechanism::Type negotiated_mech_;
 
+  // The SASL protocol name that is used for the SASL negotiation.
+  const std::string sasl_proto_name_;
+
   // Negotiation timeout deadline.
   MonoTime deadline_;
 };

http://git-wip-us.apache.org/repos/asf/kudu/blob/31d58522/src/kudu/rpc/constants.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/constants.cc b/src/kudu/rpc/constants.cc
index bcf9985..a4e024c 100644
--- a/src/kudu/rpc/constants.cc
+++ b/src/kudu/rpc/constants.cc
@@ -24,7 +24,6 @@ namespace rpc {
 
 const char* const kMagicNumber = "hrpc";
 const char* const kSaslAppName = "kudu";
-const char* const kSaslProtoName = "kudu";
 
 // NOTE: the TLS flag is dynamically added based on the local encryption
 // configuration.

http://git-wip-us.apache.org/repos/asf/kudu/blob/31d58522/src/kudu/rpc/constants.h
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/constants.h b/src/kudu/rpc/constants.h
index e179ddc..a3c7c67 100644
--- a/src/kudu/rpc/constants.h
+++ b/src/kudu/rpc/constants.h
@@ -32,9 +32,6 @@ extern const char* const kMagicNumber;
 // App name for SASL library init
 extern const char* const kSaslAppName;
 
-// Network protocol name for SASL library init
-extern const char* const kSaslProtoName;
-
 // Current version of the RPC protocol.
 static const uint32_t kCurrentRpcVersion = 9;
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/31d58522/src/kudu/rpc/messenger.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/messenger.cc b/src/kudu/rpc/messenger.cc
index ed463cf..46b144f 100644
--- a/src/kudu/rpc/messenger.cc
+++ b/src/kudu/rpc/messenger.cc
@@ -232,6 +232,7 @@ MessengerBuilder::MessengerBuilder(std::string name)
       min_negotiation_threads_(0),
       max_negotiation_threads_(4),
       coarse_timer_granularity_(MonoDelta::FromMilliseconds(100)),
+      sasl_proto_name_("kudu"),
       enable_inbound_tls_(false) {
 }
 
@@ -266,6 +267,11 @@ MessengerBuilder &MessengerBuilder::set_metric_entity(
   return *this;
 }
 
+MessengerBuilder &MessengerBuilder::set_sasl_proto_name(std::string sasl_proto_name) {
+  sasl_proto_name_ = std::move(sasl_proto_name);
+  return *this;
+}
+
 MessengerBuilder& MessengerBuilder::enable_inbound_tls() {
   enable_inbound_tls_ = true;
   return *this;
@@ -396,7 +402,7 @@ Status Messenger::AddAcceptorPool(const Sockaddr &accept_addr,
   // that everything is set up correctly. This way we'll generate errors on
   // startup rather than later on when we first receive a client connection.
   if (!FLAGS_keytab_file.empty()) {
-    RETURN_NOT_OK_PREPEND(ServerNegotiation::PreflightCheckGSSAPI(),
+    RETURN_NOT_OK_PREPEND(ServerNegotiation::PreflightCheckGSSAPI(sasl_proto_name()),
                           "GSSAPI/Kerberos not properly configured");
   }
 
@@ -491,6 +497,7 @@ Messenger::Messenger(const MessengerBuilder &bld)
     token_verifier_(new security::TokenVerifier()),
     rpcz_store_(new RpczStore()),
     metric_entity_(bld.metric_entity_),
+    sasl_proto_name_(bld.sasl_proto_name_),
     retain_self_(this) {
   for (int i = 0; i < bld.num_reactors_; i++) {
     reactors_.push_back(new Reactor(retain_self_, i, bld));

http://git-wip-us.apache.org/repos/asf/kudu/blob/31d58522/src/kudu/rpc/messenger.h
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/messenger.h b/src/kudu/rpc/messenger.h
index cf34c95..7f66ab2 100644
--- a/src/kudu/rpc/messenger.h
+++ b/src/kudu/rpc/messenger.h
@@ -121,6 +121,9 @@ class MessengerBuilder {
   // Set metric entity for use by RPC systems.
   MessengerBuilder &set_metric_entity(const scoped_refptr<MetricEntity>& metric_entity);
 
+  // Set the SASL protocol name that is used for the SASL negotiation.
+  MessengerBuilder &set_sasl_proto_name(std::string sasl_proto_name);
+
   // Configure the messenger to enable TLS encryption on inbound connections.
   MessengerBuilder& enable_inbound_tls();
 
@@ -134,6 +137,7 @@ class MessengerBuilder {
   int max_negotiation_threads_;
   MonoDelta coarse_timer_granularity_;
   scoped_refptr<MetricEntity> metric_entity_;
+  std::string sasl_proto_name_;
   bool enable_inbound_tls_;
 };
 
@@ -260,6 +264,10 @@ class Messenger {
 
   scoped_refptr<MetricEntity> metric_entity() const { return metric_entity_.get(); }
 
+  const std::string& sasl_proto_name() const {
+    return sasl_proto_name_;
+  }
+
   const scoped_refptr<RpcService> rpc_service(const std::string& service_name) const;
 
  private:
@@ -330,6 +338,9 @@ class Messenger {
 
   scoped_refptr<MetricEntity> metric_entity_;
 
+  // The SASL protocol name that is used for the SASL negotiation.
+  const std::string sasl_proto_name_;
+
   // The ownership of the Messenger object is somewhat subtle. The pointer graph
   // looks like this:
   //

http://git-wip-us.apache.org/repos/asf/kudu/blob/31d58522/src/kudu/rpc/negotiation-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/negotiation-test.cc b/src/kudu/rpc/negotiation-test.cc
index fb4f240..4158ffc 100644
--- a/src/kudu/rpc/negotiation-test.cc
+++ b/src/kudu/rpc/negotiation-test.cc
@@ -242,11 +242,13 @@ TEST_P(TestNegotiation, TestNegotiation) {
   ClientNegotiation client_negotiation(std::move(client_socket),
                                        &client_tls_context,
                                        authn_token,
-                                       desc.client.encryption);
+                                       desc.client.encryption,
+                                       "kudu");
   ServerNegotiation server_negotiation(std::move(server_socket),
                                        &server_tls_context,
                                        &token_verifier,
-                                       desc.server.encryption);
+                                       desc.server.encryption,
+                                       "kudu");
 
   // Set client and server SASL mechanisms.
   MiniKdc kdc;
@@ -1025,7 +1027,7 @@ static void RunGSSAPINegotiationServer(unique_ptr<Socket> socket,
   CHECK_OK(tls_context.Init());
   TokenVerifier token_verifier;
   ServerNegotiation server_negotiation(std::move(socket), &tls_context,
-                                       &token_verifier, RpcEncryption::OPTIONAL);
+                                       &token_verifier, RpcEncryption::OPTIONAL, "kudu");
   server_negotiation.set_server_fqdn("127.0.0.1");
   CHECK_OK(server_negotiation.EnableGSSAPI());
   post_check(server_negotiation.Negotiate(), server_negotiation);
@@ -1038,7 +1040,7 @@ static void RunGSSAPINegotiationClient(unique_ptr<Socket> conn,
   TlsContext tls_context;
   CHECK_OK(tls_context.Init());
   ClientNegotiation client_negotiation(std::move(conn), &tls_context,
-                                       boost::none, RpcEncryption::OPTIONAL);
+                                       boost::none, RpcEncryption::OPTIONAL, "kudu");
   client_negotiation.set_server_fqdn("127.0.0.1");
   CHECK_OK(client_negotiation.EnableGSSAPI());
   post_check(client_negotiation.Negotiate(), client_negotiation);
@@ -1138,7 +1140,7 @@ TEST_F(TestNegotiation, TestGSSAPIInvalidNegotiation) {
 // the preflight check passes a 0-length token.
 TEST_F(TestNegotiation, TestPreflight) {
   // Try pre-flight with no keytab.
-  Status s = ServerNegotiation::PreflightCheckGSSAPI();
+  Status s = ServerNegotiation::PreflightCheckGSSAPI("kudu");
   ASSERT_FALSE(s.ok());
 #ifndef KRB5_VERSION_LE_1_10
   ASSERT_STR_MATCHES(s.ToString(), "Key table file.*not found");
@@ -1151,11 +1153,11 @@ TEST_F(TestNegotiation, TestPreflight) {
   ASSERT_OK(kdc.CreateServiceKeytab("kudu/127.0.0.1", &kt_path));
   CHECK_ERR(setenv("KRB5_KTNAME", kt_path.c_str(), 1 /*replace*/));
 
-  ASSERT_OK(ServerNegotiation::PreflightCheckGSSAPI());
+  ASSERT_OK(ServerNegotiation::PreflightCheckGSSAPI("kudu"));
 
   // Try with an inaccessible keytab.
   CHECK_ERR(chmod(kt_path.c_str(), 0000));
-  s = ServerNegotiation::PreflightCheckGSSAPI();
+  s = ServerNegotiation::PreflightCheckGSSAPI("kudu");
   ASSERT_FALSE(s.ok());
 #ifndef KRB5_VERSION_LE_1_10
   ASSERT_STR_MATCHES(s.ToString(), "error accessing keytab: Permission denied");
@@ -1165,7 +1167,7 @@ TEST_F(TestNegotiation, TestPreflight) {
   // Try with a keytab that has the wrong credentials.
   ASSERT_OK(kdc.CreateServiceKeytab("wrong-service/127.0.0.1", &kt_path));
   CHECK_ERR(setenv("KRB5_KTNAME", kt_path.c_str(), 1 /*replace*/));
-  s = ServerNegotiation::PreflightCheckGSSAPI();
+  s = ServerNegotiation::PreflightCheckGSSAPI("kudu");
   ASSERT_FALSE(s.ok());
 #ifndef KRB5_VERSION_LE_1_10
   ASSERT_STR_MATCHES(s.ToString(), "No key table entry found matching kudu/.*");
@@ -1180,7 +1182,7 @@ static void RunTimeoutExpectingServer(unique_ptr<Socket> socket) {
   CHECK_OK(tls_context.Init());
   TokenVerifier token_verifier;
   ServerNegotiation server_negotiation(std::move(socket), &tls_context,
-                                       &token_verifier, RpcEncryption::OPTIONAL);
+                                       &token_verifier, RpcEncryption::OPTIONAL, "kudu");
   CHECK_OK(server_negotiation.EnablePlain());
   Status s = server_negotiation.Negotiate();
   ASSERT_TRUE(s.IsNetworkError()) << "Expected client to time out and close the connection. Got: "
@@ -1191,7 +1193,7 @@ static void RunTimeoutNegotiationClient(unique_ptr<Socket> sock) {
   TlsContext tls_context;
   CHECK_OK(tls_context.Init());
   ClientNegotiation client_negotiation(std::move(sock), &tls_context,
-                                       boost::none, RpcEncryption::OPTIONAL);
+                                       boost::none, RpcEncryption::OPTIONAL, "kudu");
   CHECK_OK(client_negotiation.EnablePlain("test", "test"));
   MonoTime deadline = MonoTime::Now() - MonoDelta::FromMilliseconds(100L);
   client_negotiation.set_deadline(deadline);
@@ -1212,7 +1214,7 @@ static void RunTimeoutNegotiationServer(unique_ptr<Socket> socket) {
   CHECK_OK(tls_context.Init());
   TokenVerifier token_verifier;
   ServerNegotiation server_negotiation(std::move(socket), &tls_context,
-                                       &token_verifier, RpcEncryption::OPTIONAL);
+                                       &token_verifier, RpcEncryption::OPTIONAL, "kudu");
   CHECK_OK(server_negotiation.EnablePlain());
   MonoTime deadline = MonoTime::Now() - MonoDelta::FromMilliseconds(100L);
   server_negotiation.set_deadline(deadline);
@@ -1225,7 +1227,7 @@ static void RunTimeoutExpectingClient(unique_ptr<Socket> socket) {
   TlsContext tls_context;
   CHECK_OK(tls_context.Init());
   ClientNegotiation client_negotiation(std::move(socket), &tls_context,
-                                       boost::none, RpcEncryption::OPTIONAL);
+                                       boost::none, RpcEncryption::OPTIONAL, "kudu");
   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: "

http://git-wip-us.apache.org/repos/asf/kudu/blob/31d58522/src/kudu/rpc/negotiation.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/negotiation.cc b/src/kudu/rpc/negotiation.cc
index 9013d8b..92f5e1d 100644
--- a/src/kudu/rpc/negotiation.cc
+++ b/src/kudu/rpc/negotiation.cc
@@ -178,7 +178,8 @@ static Status DoClientNegotiation(Connection* conn,
   ClientNegotiation client_negotiation(conn->release_socket(),
                                        &messenger->tls_context(),
                                        authn_token,
-                                       encryption);
+                                       encryption,
+                                       messenger->sasl_proto_name());
 
   client_negotiation.set_server_fqdn(conn->outbound_connection_id().hostname());
 
@@ -256,7 +257,8 @@ static Status DoServerNegotiation(Connection* conn,
   ServerNegotiation server_negotiation(conn->release_socket(),
                                        &messenger->tls_context(),
                                        &messenger->token_verifier(),
-                                       encryption);
+                                       encryption,
+                                       messenger->sasl_proto_name());
 
   if (authentication != RpcAuthentication::DISABLED && !FLAGS_keytab_file.empty()) {
     RETURN_NOT_OK(server_negotiation.EnableGSSAPI());

http://git-wip-us.apache.org/repos/asf/kudu/blob/31d58522/src/kudu/rpc/server_negotiation.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/server_negotiation.cc b/src/kudu/rpc/server_negotiation.cc
index 7ddcae0..6e87fb6 100644
--- a/src/kudu/rpc/server_negotiation.cc
+++ b/src/kudu/rpc/server_negotiation.cc
@@ -133,7 +133,8 @@ static int ServerNegotiationPlainAuthCb(sasl_conn_t* conn,
 ServerNegotiation::ServerNegotiation(unique_ptr<Socket> socket,
                                      const security::TlsContext* tls_context,
                                      const security::TokenVerifier* token_verifier,
-                                     RpcEncryption encryption)
+                                     RpcEncryption encryption,
+                                     std::string sasl_proto_name)
     : socket_(std::move(socket)),
       helper_(SaslHelper::SERVER),
       tls_context_(tls_context),
@@ -142,6 +143,7 @@ ServerNegotiation::ServerNegotiation(unique_ptr<Socket> socket,
       token_verifier_(token_verifier),
       negotiated_authn_(AuthenticationType::INVALID),
       negotiated_mech_(SaslMechanism::INVALID),
+      sasl_proto_name_(std::move(sasl_proto_name)),
       deadline_(MonoTime::Max()) {
   callbacks_.push_back(SaslBuildCallback(SASL_CB_GETOPT,
       reinterpret_cast<int (*)()>(&ServerNegotiationGetoptCb), this));
@@ -263,7 +265,7 @@ Status ServerNegotiation::Negotiate() {
   return Status::OK();
 }
 
-Status ServerNegotiation::PreflightCheckGSSAPI() {
+Status ServerNegotiation::PreflightCheckGSSAPI(const std::string& sasl_proto_name) {
   // TODO(todd): the error messages that come from this function on el6
   // are relatively useless due to the following krb5 bug:
   // http://krbdev.mit.edu/rt/Ticket/Display.html?id=6973
@@ -275,7 +277,8 @@ 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, nullptr, nullptr, RpcEncryption::OPTIONAL);
+  ServerNegotiation server(
+      nullptr, nullptr, nullptr, RpcEncryption::OPTIONAL, sasl_proto_name);
   Status s = server.EnableGSSAPI();
   if (!s.ok()) {
     return Status::RuntimeError(s.message());
@@ -382,7 +385,7 @@ Status ServerNegotiation::InitSaslServer() {
   RETURN_NOT_OK_PREPEND(WrapSaslCall(nullptr /* no conn */, [&]() {
       return sasl_server_new(
           // Registered name of the service using SASL. Required.
-          kSaslProtoName,
+          sasl_proto_name_.c_str(),
           // The fully qualified domain name of this server.
           helper_.server_fqdn(),
           // Permits multiple user realms on server. NULL == use default.

http://git-wip-us.apache.org/repos/asf/kudu/blob/31d58522/src/kudu/rpc/server_negotiation.h
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/server_negotiation.h b/src/kudu/rpc/server_negotiation.h
index 18d4e2a..2255274 100644
--- a/src/kudu/rpc/server_negotiation.h
+++ b/src/kudu/rpc/server_negotiation.h
@@ -64,7 +64,8 @@ class ServerNegotiation {
   ServerNegotiation(std::unique_ptr<Socket> socket,
                     const security::TlsContext* tls_context,
                     const security::TokenVerifier* token_verifier,
-                    RpcEncryption encryption);
+                    RpcEncryption encryption,
+                    std::string sasl_proto_name);
 
   // Enable PLAIN authentication.
   // Despite PLAIN authentication taking a username and password, we disregard
@@ -146,7 +147,7 @@ class ServerNegotiation {
 
   // Perform a "pre-flight check" that everything required to act as a Kerberos
   // server is properly set up.
-  static Status PreflightCheckGSSAPI() WARN_UNUSED_RESULT;
+  static Status PreflightCheckGSSAPI(const std::string& sasl_proto_name) WARN_UNUSED_RESULT;
 
  private:
 
@@ -246,6 +247,9 @@ class ServerNegotiation {
   // authentication type is SASL.
   SaslMechanism::Type negotiated_mech_;
 
+  // The SASL protocol name that is used for the SASL negotiation.
+  const std::string sasl_proto_name_;
+
   // Negotiation timeout deadline.
   MonoTime deadline_;
 };