You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by al...@apache.org on 2018/01/18 05:32:24 UTC

kudu git commit: [rpc] don't issue authn tokens over non-confidential connections

Repository: kudu
Updated Branches:
  refs/heads/master d3165ed2b -> 1f346a1be


[rpc] don't issue authn tokens over non-confidential connections

With this patch, master will not issue authn tokens to clients over a
non-confidential connection.  The 'confidentiality' property is based
on the features of the negotiated connection and stored into the
RpcContext object, accessible via the RpcContext::is_confidential()
method.

Change-Id: Ie31aa492bcc460dbd43975bccfe571354f3bf885
Reviewed-on: http://gerrit.cloudera.org:8080/9025
Tested-by: Kudu Jenkins
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/1f346a1b
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/1f346a1b
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/1f346a1b

Branch: refs/heads/master
Commit: 1f346a1bec3db99877a4ace0d3682e5341bec9c6
Parents: d3165ed
Author: Alexey Serbin <as...@cloudera.com>
Authored: Thu Jan 11 14:55:20 2018 -0800
Committer: Alexey Serbin <as...@cloudera.com>
Committed: Thu Jan 18 05:31:15 2018 +0000

----------------------------------------------------------------------
 src/kudu/integration-tests/security-itest.cc | 105 ++++++++++++++++++++++
 src/kudu/master/master_service.cc            |   7 +-
 src/kudu/rpc/connection.cc                   |   5 ++
 src/kudu/rpc/connection.h                    |  12 +++
 src/kudu/rpc/negotiation.cc                  |   4 +
 src/kudu/rpc/rpc_context.cc                  |   5 ++
 src/kudu/rpc/rpc_context.h                   |   7 ++
 7 files changed, 143 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/1f346a1b/src/kudu/integration-tests/security-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/security-itest.cc b/src/kudu/integration-tests/security-itest.cc
index 16c0175..6fb76bf 100644
--- a/src/kudu/integration-tests/security-itest.cc
+++ b/src/kudu/integration-tests/security-itest.cc
@@ -22,11 +22,13 @@
 #include <string>
 #include <vector>
 
+#include <gflags/gflags_declare.h>
 #include <glog/logging.h>
 #include <gtest/gtest.h>
 
 #include "kudu/client/client-test-util.h"
 #include "kudu/client/client.h"
+#include "kudu/client/client.pb.h"
 #include "kudu/client/schema.h"
 #include "kudu/client/shared_ptr.h"
 #include "kudu/client/write_op.h"
@@ -37,9 +39,11 @@
 #include "kudu/master/master.pb.h"
 #include "kudu/master/master.proxy.h"
 #include "kudu/mini-cluster/external_mini_cluster.h"
+#include "kudu/mini-cluster/mini_cluster.h"
 #include "kudu/rpc/messenger.h"
 #include "kudu/rpc/rpc_controller.h"
 #include "kudu/security/test/mini_kdc.h"
+#include "kudu/security/token.pb.h"
 #include "kudu/server/server_base.pb.h"
 #include "kudu/server/server_base.proxy.h"
 #include "kudu/tablet/key_value_test_schema.h"
@@ -52,6 +56,8 @@
 #include "kudu/util/test_macros.h"
 #include "kudu/util/test_util.h"
 
+DECLARE_string(local_ip_for_outbound_sockets);
+
 using kudu::client::KuduClient;
 using kudu::client::KuduInsert;
 using kudu::client::KuduSchema;
@@ -304,4 +310,103 @@ TEST_F(SecurityITest, TestWorldReadablePrivateKey) {
       credentials_name));
 }
 
+struct AuthTokenIssuingTestParams {
+  const ExternalMiniCluster::BindMode bind_mode;
+  const string rpc_authentication;
+  const string rpc_encryption;
+  const bool rpc_encrypt_loopback_connections;
+  const bool authn_token_present;
+};
+class AuthTokenIssuingTest :
+    public SecurityITest,
+    public ::testing::WithParamInterface<AuthTokenIssuingTestParams> {
+};
+INSTANTIATE_TEST_CASE_P(, AuthTokenIssuingTest, ::testing::ValuesIn(
+    vector<AuthTokenIssuingTestParams>{
+      { ExternalMiniCluster::BindMode::LOOPBACK, "required", "required", true,  true,  },
+      { ExternalMiniCluster::BindMode::LOOPBACK, "required", "required", false, true,  },
+      //ExternalMiniCluster::BindMode::LOOPBACK, "required", "disabled": non-acceptable
+      //ExternalMiniCluster::BindMode::LOOPBACK, "required", "disabled": non-acceptable
+      { ExternalMiniCluster::BindMode::LOOPBACK, "disabled", "required", true,  true,  },
+      { ExternalMiniCluster::BindMode::LOOPBACK, "disabled", "required", false, true,  },
+      { ExternalMiniCluster::BindMode::LOOPBACK, "disabled", "disabled", true,  false, },
+      { ExternalMiniCluster::BindMode::LOOPBACK, "disabled", "disabled", false, true,  },
+#if defined(__linux__)
+      { ExternalMiniCluster::BindMode::UNIQUE_LOOPBACK, "required", "required", true,  true,  },
+      { ExternalMiniCluster::BindMode::UNIQUE_LOOPBACK, "required", "required", false, true,  },
+      //ExternalMiniCluster::BindMode::UNIQUE_LOOPBACK, "required", "disabled": non-acceptable
+      //ExternalMiniCluster::BindMode::UNIQUE_LOOPBACK, "required", "disabled": non-acceptable
+      { ExternalMiniCluster::BindMode::UNIQUE_LOOPBACK, "disabled", "required", true,  true,  },
+      { ExternalMiniCluster::BindMode::UNIQUE_LOOPBACK, "disabled", "required", false, true,  },
+      { ExternalMiniCluster::BindMode::UNIQUE_LOOPBACK, "disabled", "disabled", true,  false, },
+      { ExternalMiniCluster::BindMode::UNIQUE_LOOPBACK, "disabled", "disabled", false, false, },
+#endif
+    }
+));
+
+// Verify how master issues authn tokens to clients. Master sends authn tokens
+// to clients upon call of the ConnectToMaster() RPC. The master's behavior
+// must depend on whether the connection to the client is confidential or not.
+TEST_P(AuthTokenIssuingTest, ChannelConfidentiality) {
+  cluster_opts_.num_masters = 1;
+  cluster_opts_.num_tablet_servers = 0;
+  // --user-acl: just restoring back the default setting.
+  cluster_opts_.extra_master_flags.emplace_back("--user-acl=*");
+
+  const auto& params = GetParam();
+  cluster_opts_.bind_mode = params.bind_mode;
+  cluster_opts_.extra_master_flags.emplace_back(
+      Substitute("--rpc-authentication=$0", params.rpc_authentication));
+  cluster_opts_.extra_master_flags.emplace_back(
+      Substitute("--rpc-encryption=$0", params.rpc_encryption));
+  cluster_opts_.extra_master_flags.emplace_back(
+      Substitute("--rpc_encrypt_loopback_connections=$0",
+                 params.rpc_encrypt_loopback_connections));
+  ASSERT_OK(StartCluster());
+
+  // Make sure the client always connects from the standard loopback address.
+  // This is crucial when the master is running with UNIQUE_LOOPBACK mode: the
+  // test scenario expects the client connects from other than 127.0.0.1 address
+  // so the connection is not considered a 'loopback' one.
+  FLAGS_local_ip_for_outbound_sockets = "127.0.0.1";
+
+  // Current implementation of MasterServiceImpl::ConnectToMaster() allows to
+  // get a success response without proper security information in case if the
+  // master hasn't been established as a leader yet. As a temporary workaround,
+  // make sure the master is sending back the necessary info before going any
+  // further with the scenario which is sensitive to that issue.
+  //
+  // TODO(aserbin): fix the issue with MasterServiceImpl::ConnectToMaster()
+  //                and remove this ASSERT_EVENTUALLY() block.
+  ASSERT_EVENTUALLY([&] {
+    client::sp::shared_ptr<KuduClient> client;
+    ASSERT_OK(cluster_->CreateClient(nullptr, &client));
+
+    string authn_creds;
+    ASSERT_OK(client->ExportAuthenticationCredentials(&authn_creds));
+    client::AuthenticationCredentialsPB pb;
+    ASSERT_TRUE(pb.ParseFromString(authn_creds));
+    ASSERT_GE(pb.ca_cert_ders_size(), 1);
+  });
+
+  // In current implementation, KuduClientBuilder calls ConnectToCluster() on
+  // the newly created instance of the KuduClient.
+  client::sp::shared_ptr<KuduClient> client;
+  ASSERT_OK(cluster_->CreateClient(nullptr, &client));
+
+  string authn_creds;
+  ASSERT_OK(client->ExportAuthenticationCredentials(&authn_creds));
+  client::AuthenticationCredentialsPB pb;
+  ASSERT_TRUE(pb.ParseFromString(authn_creds));
+  ASSERT_EQ(params.authn_token_present, pb.has_authn_token());
+
+  if (pb.has_authn_token()) {
+    // If authn token is present, then check it for consistency.
+    const auto& t = pb.authn_token();
+    EXPECT_TRUE(t.has_token_data());
+    EXPECT_TRUE(t.has_signature());
+    EXPECT_TRUE(t.has_signing_key_seq_num());
+  }
+}
+
 } // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/1f346a1b/src/kudu/master/master_service.cc
----------------------------------------------------------------------
diff --git a/src/kudu/master/master_service.cc b/src/kudu/master/master_service.cc
index 6dc8965..2e50fe9 100644
--- a/src/kudu/master/master_service.cc
+++ b/src/kudu/master/master_service.cc
@@ -480,8 +480,11 @@ void MasterServiceImpl::ConnectToMaster(const ConnectToMasterRequestPB* /*req*/,
     resp->add_ca_cert_der(server_->cert_authority()->ca_cert_der());
 
     // Issue an authentication token for the caller, unless they are
-    // already using a token to authenticate.
-    if (rpc->remote_user().authenticated_by() != rpc::RemoteUser::AUTHN_TOKEN) {
+    // already using a token to authenticate or haven't been authenticated
+    // by other means. Don't issue a token if it's about to travel back to the
+    // client over a non-confidential channel.
+    if (rpc->is_confidential() &&
+        rpc->remote_user().authenticated_by() != rpc::RemoteUser::AUTHN_TOKEN) {
       SignedTokenPB authn_token;
       Status s = server_->token_signer()->GenerateAuthnToken(
           rpc->remote_user().username(),

http://git-wip-us.apache.org/repos/asf/kudu/blob/1f346a1b/src/kudu/rpc/connection.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/connection.cc b/src/kudu/rpc/connection.cc
index a28eddb..857649b 100644
--- a/src/kudu/rpc/connection.cc
+++ b/src/kudu/rpc/connection.cc
@@ -78,6 +78,7 @@ Connection::Connection(ReactorThread *reactor_thread,
       next_call_id_(1),
       credentials_policy_(policy),
       negotiation_complete_(false),
+      is_confidential_(false),
       scheduled_for_shutdown_(false) {
 }
 
@@ -478,6 +479,10 @@ void Connection::QueueResponseForCall(gscoped_ptr<InboundCall> call) {
   reactor_thread_->reactor()->ScheduleReactorTask(task);
 }
 
+void Connection::set_confidential(bool is_confidential) {
+  is_confidential_ = is_confidential;
+}
+
 bool Connection::SatisfiesCredentialsPolicy(CredentialsPolicy policy) const {
   DCHECK_EQ(direction_, CLIENT);
   return (policy == CredentialsPolicy::ANY_CREDENTIALS) ||

http://git-wip-us.apache.org/repos/asf/kudu/blob/1f346a1b/src/kudu/rpc/connection.h
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/connection.h b/src/kudu/rpc/connection.h
index 4e44132..a1f5911 100644
--- a/src/kudu/rpc/connection.h
+++ b/src/kudu/rpc/connection.h
@@ -149,6 +149,13 @@ class Connection : public RefCountedThreadSafe<Connection> {
     return *outbound_connection_id_;
   }
 
+  bool is_confidential() const {
+    return is_confidential_;
+  }
+
+  // Set/unset the 'confidentiality' property for this connection.
+  void set_confidential(bool is_confidential);
+
   // Credentials policy to start connection negotiation.
   CredentialsPolicy credentials_policy() const { return credentials_policy_; }
 
@@ -364,6 +371,11 @@ class Connection : public RefCountedThreadSafe<Connection> {
   // Whether we completed connection negotiation.
   bool negotiation_complete_;
 
+  // Whether it's OK to pass confidential information over the connection.
+  // For example, an encrypted (but not necessarily authenticated) connection
+  // is considered confidential.
+  bool is_confidential_;
+
   // Whether the connection is scheduled for shutdown.
   bool scheduled_for_shutdown_;
 };

http://git-wip-us.apache.org/repos/asf/kudu/blob/1f346a1b/src/kudu/rpc/negotiation.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/negotiation.cc b/src/kudu/rpc/negotiation.cc
index d85ec5d..31e0b33 100644
--- a/src/kudu/rpc/negotiation.cc
+++ b/src/kudu/rpc/negotiation.cc
@@ -220,6 +220,8 @@ static Status DoClientNegotiation(Connection* conn,
   // Transfer the negotiated socket and state back to the connection.
   conn->adopt_socket(client_negotiation.release_socket());
   conn->set_remote_features(client_negotiation.take_server_features());
+  conn->set_confidential(client_negotiation.tls_negotiated() ||
+      (conn->socket()->IsLoopbackConnection() && !FLAGS_rpc_encrypt_loopback_connections));
 
   // Sanity check: if no authn token was supplied as user credentials,
   // the negotiated authentication type cannot be AuthenticationType::TOKEN.
@@ -274,6 +276,8 @@ static Status DoServerNegotiation(Connection* conn,
   conn->adopt_socket(server_negotiation.release_socket());
   conn->set_remote_features(server_negotiation.take_client_features());
   conn->set_remote_user(server_negotiation.take_authenticated_user());
+  conn->set_confidential(server_negotiation.tls_negotiated() ||
+      (conn->socket()->IsLoopbackConnection() && !FLAGS_rpc_encrypt_loopback_connections));
 
   return Status::OK();
 }

http://git-wip-us.apache.org/repos/asf/kudu/blob/1f346a1b/src/kudu/rpc/rpc_context.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/rpc_context.cc b/src/kudu/rpc/rpc_context.cc
index 3e33cbb..123d21f 100644
--- a/src/kudu/rpc/rpc_context.cc
+++ b/src/kudu/rpc/rpc_context.cc
@@ -24,6 +24,7 @@
 #include <glog/logging.h>
 #include <google/protobuf/message.h>
 
+#include "kudu/rpc/connection.h"
 #include "kudu/rpc/inbound_call.h"
 #include "kudu/rpc/remote_method.h"
 #include "kudu/rpc/remote_user.h"
@@ -155,6 +156,10 @@ const RemoteUser& RpcContext::remote_user() const {
   return call_->remote_user();
 }
 
+bool RpcContext::is_confidential() const {
+  return call_->connection()->is_confidential();
+}
+
 void RpcContext::DiscardTransfer() {
   call_->DiscardTransfer();
 }

http://git-wip-us.apache.org/repos/asf/kudu/blob/1f346a1b/src/kudu/rpc/rpc_context.h
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/rpc_context.h b/src/kudu/rpc/rpc_context.h
index 201cc84..a34c1a1 100644
--- a/src/kudu/rpc/rpc_context.h
+++ b/src/kudu/rpc/rpc_context.h
@@ -168,6 +168,13 @@ class RpcContext {
   // Return the identity of remote user who made this call.
   const RemoteUser& remote_user() const;
 
+  // Whether it's OK to pass confidential information between the client and the
+  // server in the context of the RPC call being handled.  In real world, this
+  // translates into properties of the connection between the client and the
+  // server. For example, this methods returns 'true' for a call over an
+  // encrypted connection.
+  bool is_confidential() const;
+
   // Discards the memory associated with the inbound call's payload. All previously
   // obtained sidecar slices will be invalidated by this call. It is an error to call
   // GetInboundSidecar() after this method. request_pb() remains valid.