Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id CF009200C23 for ; Tue, 7 Feb 2017 19:59:13 +0100 (CET) Received: by cust-asf.ponee.io (Postfix) id CDB2F160B3E; Tue, 7 Feb 2017 18:59:13 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id ABD59160B32 for ; Tue, 7 Feb 2017 19:59:12 +0100 (CET) Received: (qmail 67375 invoked by uid 500); 7 Feb 2017 18:59:11 -0000 Mailing-List: contact commits-help@kudu.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@kudu.apache.org Delivered-To: mailing list commits@kudu.apache.org Received: (qmail 67361 invoked by uid 99); 7 Feb 2017 18:59:11 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 07 Feb 2017 18:59:11 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id BC7A1DFB0E; Tue, 7 Feb 2017 18:59:11 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: danburkert@apache.org To: commits@kudu.apache.org Date: Tue, 07 Feb 2017 18:59:11 -0000 Message-Id: X-Mailer: ASF-Git Admin Mailer Subject: [1/3] kudu git commit: master: issue authentication tokens and CA certs to clients archived-at: Tue, 07 Feb 2017 18:59:14 -0000 Repository: kudu Updated Branches: refs/heads/master d47c13872 -> 8971bc535 master: issue authentication tokens and CA certs to clients This adds the code to the ConnectToMaster RPC to issue an authentication token for the connecting user, as well as to send it the current master CA cert. The client side currently ignores both things, but a new unit test verifies that they are getting properly set. As with most of these early integration patches, there are plenty of TODOs about integrating with the catalog manager, etc, but this should be enough to unblock other client-side work of propagating the CA and tokens into the RPC system. Change-Id: I5969b8e125633b3b14364b98c0d0a992b162f302 Reviewed-on: http://gerrit.cloudera.org:8080/5871 Tested-by: Kudu Jenkins Reviewed-by: Alexey Serbin Project: http://git-wip-us.apache.org/repos/asf/kudu/repo Commit: http://git-wip-us.apache.org/repos/asf/kudu/commit/7d434601 Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/7d434601 Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/7d434601 Branch: refs/heads/master Commit: 7d43460109899af9ef40eb2bebb6f532f5dd0055 Parents: d47c138 Author: Todd Lipcon Authored: Wed Feb 1 22:38:25 2017 -0800 Committer: Todd Lipcon Committed: Tue Feb 7 18:36:16 2017 +0000 ---------------------------------------------------------------------- src/kudu/master/CMakeLists.txt | 2 + src/kudu/master/authn_token_manager.cc | 98 +++++++++++++++++++++++++++ src/kudu/master/authn_token_manager.h | 50 ++++++++++++++ src/kudu/master/master-test.cc | 21 ++++++ src/kudu/master/master.cc | 7 ++ src/kudu/master/master.h | 4 ++ src/kudu/master/master.proto | 15 ++++ src/kudu/master/master_cert_authority.cc | 7 +- src/kudu/master/master_cert_authority.h | 15 ++++ src/kudu/master/master_service.cc | 36 +++++++++- src/kudu/rpc/outbound_call.h | 1 + src/kudu/security/token.proto | 5 +- 12 files changed, 257 insertions(+), 4 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/kudu/blob/7d434601/src/kudu/master/CMakeLists.txt ---------------------------------------------------------------------- diff --git a/src/kudu/master/CMakeLists.txt b/src/kudu/master/CMakeLists.txt index 3afc7e6..673132e 100644 --- a/src/kudu/master/CMakeLists.txt +++ b/src/kudu/master/CMakeLists.txt @@ -33,6 +33,7 @@ ADD_EXPORTABLE_LIBRARY(master_proto NONLINK_DEPS ${MASTER_KRPC_TGTS}) set(MASTER_SRCS + authn_token_manager.cc catalog_manager.cc master.cc master_cert_authority.cc @@ -57,6 +58,7 @@ target_link_libraries(master server_common server_process tablet + token_proto tserver tserver_service_proto) http://git-wip-us.apache.org/repos/asf/kudu/blob/7d434601/src/kudu/master/authn_token_manager.cc ---------------------------------------------------------------------- diff --git a/src/kudu/master/authn_token_manager.cc b/src/kudu/master/authn_token_manager.cc new file mode 100644 index 0000000..b9b8306 --- /dev/null +++ b/src/kudu/master/authn_token_manager.cc @@ -0,0 +1,98 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include "kudu/master/authn_token_manager.h" + +#include +#include + +#include + +#include "kudu/gutil/walltime.h" +#include "kudu/security/token.pb.h" +#include "kudu/security/token_signer.h" +#include "kudu/util/flag_tags.h" +#include "kudu/util/status.h" + +DEFINE_int64(authn_token_validity_seconds, 120, + "Period of time for which an issued authentication token is valid."); +// TODO(PKI): docs for what actual effect this has, given we don't support +// token renewal. +// TODO(PKI): this is set extremely low, so that we don't forget to come back to +// this and add rolling and refetching code. +TAG_FLAG(authn_token_validity_seconds, experimental); + + +using kudu::security::AuthnTokenPB; +using kudu::security::SignedTokenPB; +using kudu::security::TokenPB; +using kudu::security::TokenSigner; +using std::unique_ptr; +using std::string; + +namespace kudu { +namespace master { + +AuthnTokenManager::AuthnTokenManager() { +} + +AuthnTokenManager::~AuthnTokenManager() { +} + +Status AuthnTokenManager::Init(int64_t next_tsk_seq_num) { + CHECK(!signer_); + unique_ptr signer(new TokenSigner(next_tsk_seq_num)); + + // Roll twice at startup. See TokenSigner class documentation for reasoning. + RETURN_NOT_OK(signer->RotateSigningKey()); + RETURN_NOT_OK(signer->RotateSigningKey()); + + // TODO(PKI): need to persist the public keys every time we roll. There's + // a bit of subtlety here: we shouldn't start exporting a key until it is + // properly persisted. Perhaps need some refactor, so we can do: + // 1) generate a new TSK + // 2) try to write the public portion to system table (keep in mind we could lose + // leadership here) + // 3) pass it back to the TokenSigner as successful? + + // TODO(PKI): manage a thread which periodically rolls the TSK. Otherwise + // we'll die after some number of days (whatever the validity is). + + signer_ = std::move(signer); + return Status::OK(); +} + +Status AuthnTokenManager::GenerateToken(string username, + SignedTokenPB* signed_token) { + TokenPB token; + + token.set_expire_unix_epoch_seconds( + WallTime_Now() + FLAGS_authn_token_validity_seconds); + AuthnTokenPB* authn = token.mutable_authn(); + authn->mutable_username()->assign(std::move(username)); + SignedTokenPB ret; + + if (!token.SerializeToString(ret.mutable_token_data())) { + return Status::RuntimeError("could not serialize authn token"); + } + RETURN_NOT_OK_PREPEND(signer_->SignToken(&ret), "could not sign authn token"); + signed_token->Swap(&ret); + return Status::OK(); +} + +} // namespace master +} // namespace kudu http://git-wip-us.apache.org/repos/asf/kudu/blob/7d434601/src/kudu/master/authn_token_manager.h ---------------------------------------------------------------------- diff --git a/src/kudu/master/authn_token_manager.h b/src/kudu/master/authn_token_manager.h new file mode 100644 index 0000000..701ee91 --- /dev/null +++ b/src/kudu/master/authn_token_manager.h @@ -0,0 +1,50 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. +#pragma once + +#include +#include + +#include "kudu/gutil/macros.h" + +namespace kudu { +class Status; + +namespace security { +class SignedTokenPB; +class TokenSigner; +} // namespace security + +namespace master { + +class AuthnTokenManager { + public: + AuthnTokenManager(); + ~AuthnTokenManager(); + + Status Init(int64_t next_tsk_seq_num); + + Status GenerateToken(std::string username, + security::SignedTokenPB* signed_token); + + private: + std::unique_ptr signer_; + DISALLOW_COPY_AND_ASSIGN(AuthnTokenManager); +}; + +} // namespace master +} // namespace kudu http://git-wip-us.apache.org/repos/asf/kudu/blob/7d434601/src/kudu/master/master-test.cc ---------------------------------------------------------------------- diff --git a/src/kudu/master/master-test.cc b/src/kudu/master/master-test.cc index c5979e6..8e6c8ee 100644 --- a/src/kudu/master/master-test.cc +++ b/src/kudu/master/master-test.cc @@ -1285,5 +1285,26 @@ TEST_F(MasterTest, TestConcurrentCreateAndRenameOfSameTable) { ASSERT_OK(verifier.Verify()); } +// Unit tests for the ConnectToMaster() RPC: +// should issue authentication tokens and the master CA cert. +TEST_F(MasterTest, TestConnectToMaster) { + ConnectToMasterRequestPB req; + ConnectToMasterResponsePB resp; + RpcController rpc; + ASSERT_OK(proxy_->ConnectToMaster(req, &resp, &rpc)); + SCOPED_TRACE(resp.DebugString()); + + EXPECT_EQ(consensus::LEADER, resp.role()) << "should be leader"; + ASSERT_EQ(1, resp.ca_cert_der_size()) << "should have one cert"; + EXPECT_GT(resp.ca_cert_der(0).size(), 100) << "CA cert should be at least 100 bytes"; + ASSERT_TRUE(resp.has_authn_token()) << "should return an authn token"; + EXPECT_EQ(256, resp.authn_token().signature().size()); + EXPECT_EQ(1, resp.authn_token().signing_key_seq_num()); + + security::TokenPB token; + ASSERT_TRUE(token.ParseFromString(resp.authn_token().token_data())); + ASSERT_TRUE(token.authn().has_username()); +} + } // namespace master } // namespace kudu http://git-wip-us.apache.org/repos/asf/kudu/blob/7d434601/src/kudu/master/master.cc ---------------------------------------------------------------------- diff --git a/src/kudu/master/master.cc b/src/kudu/master/master.cc index 5fb45c2..6cfe50c 100644 --- a/src/kudu/master/master.cc +++ b/src/kudu/master/master.cc @@ -27,6 +27,7 @@ #include "kudu/cfile/block_cache.h" #include "kudu/common/wire_protocol.h" #include "kudu/gutil/strings/substitute.h" +#include "kudu/master/authn_token_manager.h" #include "kudu/master/catalog_manager.h" #include "kudu/master/master_cert_authority.h" #include "kudu/master/master_service.h" @@ -102,6 +103,12 @@ Status Master::Init() { // becomes a leader. cert_authority_.reset(new MasterCertAuthority(fs_manager_->uuid())); + // TODO(PKI): this also will need to be wired together with CatalogManager + // soon, including initializing the token manager with the proper next + // sequence number. + authn_token_manager_.reset(new AuthnTokenManager()); + RETURN_NOT_OK(authn_token_manager_->Init(1)); + state_ = kInitialized; return Status::OK(); } http://git-wip-us.apache.org/repos/asf/kudu/blob/7d434601/src/kudu/master/master.h ---------------------------------------------------------------------- diff --git a/src/kudu/master/master.h b/src/kudu/master/master.h index 0c1fceb..365c58b 100644 --- a/src/kudu/master/master.h +++ b/src/kudu/master/master.h @@ -46,6 +46,7 @@ class ServicePool; namespace master { +class AuthnTokenManager; class CatalogManager; class MasterCertAuthority; class MasterPathHandlers; @@ -77,6 +78,8 @@ class Master : public server::ServerBase { MasterCertAuthority* cert_authority() { return cert_authority_.get(); } + AuthnTokenManager* authn_token_manager() { return authn_token_manager_.get(); } + TSManager* ts_manager() { return ts_manager_.get(); } CatalogManager* catalog_manager() { return catalog_manager_.get(); } @@ -122,6 +125,7 @@ class Master : public server::ServerBase { MasterState state_; std::unique_ptr cert_authority_; + std::unique_ptr authn_token_manager_; gscoped_ptr ts_manager_; gscoped_ptr catalog_manager_; gscoped_ptr path_handlers_; http://git-wip-us.apache.org/repos/asf/kudu/blob/7d434601/src/kudu/master/master.proto ---------------------------------------------------------------------- diff --git a/src/kudu/master/master.proto b/src/kudu/master/master.proto index 3ab3146..e5f5d7e 100644 --- a/src/kudu/master/master.proto +++ b/src/kudu/master/master.proto @@ -21,6 +21,7 @@ option java_package = "org.apache.kudu.master"; import "kudu/common/common.proto"; import "kudu/common/wire_protocol.proto"; import "kudu/consensus/metadata.proto"; +import "kudu/security/token.proto"; import "kudu/tablet/metadata.proto"; import "kudu/util/pb_util.proto"; @@ -557,6 +558,10 @@ message GetTableSchemaResponsePB { } message ConnectToMasterRequestPB { + // TODO(PKI): should the client specify whether it wants an authn token and CA + // information or not? + // Or should the server always send this info back, even if the client already has + // what it needs? } message ConnectToMasterResponsePB { @@ -565,6 +570,16 @@ message ConnectToMasterResponsePB { // The current role of the master. optional consensus.RaftPeerPB.Role role = 2; + + // Any CA certs used by the cluster. Currently the master only uses + // one cert, but we may support rolling this cert in the future, so + // clients should add all returned certs to their trusted CA list. + repeated bytes ca_cert_der = 3; + + // If the client requested an authentication token, and security is + // enabled on the cluster, the master returns a signed authn token. + // TODO(PKI): implement me! + optional security.SignedTokenPB authn_token = 4; } // ============================================================================ http://git-wip-us.apache.org/repos/asf/kudu/blob/7d434601/src/kudu/master/master_cert_authority.cc ---------------------------------------------------------------------- diff --git a/src/kudu/master/master_cert_authority.cc b/src/kudu/master/master_cert_authority.cc index 36fe1c7..83d9fe0 100644 --- a/src/kudu/master/master_cert_authority.cc +++ b/src/kudu/master/master_cert_authority.cc @@ -88,8 +88,14 @@ Status MasterCertAuthority::Init(unique_ptr key, unique_ptr cert) { CHECK(key); CHECK(cert); + + // Cache the exported DER-format cert. + string ca_cert_der; + RETURN_NOT_OK(cert->ToString(&ca_cert_der, security::DataFormat::DER)); + ca_private_key_ = std::move(key); ca_cert_ = std::move(cert); + ca_cert_der_ = std::move(ca_cert_der); return Status::OK(); } @@ -115,6 +121,5 @@ Status MasterCertAuthority::SignServerCSR(const string& csr_der, string* cert_de return Status::OK(); } - } // namespace master } // namespace kudu http://git-wip-us.apache.org/repos/asf/kudu/blob/7d434601/src/kudu/master/master_cert_authority.h ---------------------------------------------------------------------- diff --git a/src/kudu/master/master_cert_authority.h b/src/kudu/master/master_cert_authority.h index ca70163..4e299d1 100644 --- a/src/kudu/master/master_cert_authority.h +++ b/src/kudu/master/master_cert_authority.h @@ -19,7 +19,10 @@ #include #include +#include + #include "kudu/gutil/macros.h" +#include "kudu/util/status.h" namespace kudu { @@ -77,6 +80,14 @@ class MasterCertAuthority { // to keep the internal state consistent. Status SignServerCSR(const std::string& csr_der, std::string* cert_der); + // Export the current CA certificate in DER format. + // + // This can be sent to participants in the cluster so they can add it to + // their trust stores. + const std::string& ca_cert_der() const { + CHECK(ca_cert_) << "must Init()"; + return ca_cert_der_; + } private: friend class ::kudu::master::MasterCertAuthorityTest; // The UUID of the master. This is used as a field in the certificate. @@ -85,6 +96,10 @@ class MasterCertAuthority { std::unique_ptr ca_private_key_; std::unique_ptr ca_cert_; + // Cached copy of the CA cert encoded in DER format. This is requested + // by any connecting client, so the cache avoids conversion overhead. + std::string ca_cert_der_; + DISALLOW_COPY_AND_ASSIGN(MasterCertAuthority); }; http://git-wip-us.apache.org/repos/asf/kudu/blob/7d434601/src/kudu/master/master_service.cc ---------------------------------------------------------------------- diff --git a/src/kudu/master/master_service.cc b/src/kudu/master/master_service.cc index c153d76..b57a986 100644 --- a/src/kudu/master/master_service.cc +++ b/src/kudu/master/master_service.cc @@ -23,12 +23,14 @@ #include #include "kudu/common/wire_protocol.h" +#include "kudu/master/authn_token_manager.h" #include "kudu/master/catalog_manager.h" #include "kudu/master/master.h" #include "kudu/master/master_cert_authority.h" #include "kudu/master/ts_descriptor.h" #include "kudu/master/ts_manager.h" #include "kudu/rpc/rpc_context.h" +#include "kudu/rpc/outbound_call.h" // for UserCredentials #include "kudu/server/webserver.h" #include "kudu/util/flag_tags.h" #include "kudu/util/pb_util.h" @@ -46,6 +48,8 @@ DEFINE_bool(master_support_connect_to_master_rpc, true, TAG_FLAG(master_support_connect_to_master_rpc, unsafe); TAG_FLAG(master_support_connect_to_master_rpc, hidden); +using kudu::security::SignedTokenPB; + namespace kudu { namespace master { @@ -355,8 +359,36 @@ void MasterServiceImpl::ConnectToMaster(const ConnectToMasterRequestPB* /*req*/, if (!l.CheckIsInitializedOrRespond(resp, rpc)) { return; } - resp->set_role(server_->catalog_manager()->Role()); - + auto role = server_->catalog_manager()->Role(); + resp->set_role(role); + + if (l.leader_status().ok()) { + // TODO(PKI): it seems there is some window when 'role' is LEADER but + // in fact we aren't done initializing (and we don't have a CA cert). + // In that case, if we respond with the 'LEADER' role to a client, but + // don't pass back the CA cert, then the client won't be able to trust + // anyone... seems like a potential race bug for clients who connect + // exactly as the leader is changing. + resp->add_ca_cert_der(server_->cert_authority()->ca_cert_der()); + + // Issue an authentication token for the caller. + // TODO(PKI): we should probably only issue a token if the client is + // authenticated by kerberos, and not by another token. Otherwise we're + // essentially allowing unlimited renewal, which is probably not what + // we want. + SignedTokenPB authn_token; + Status s = server_->authn_token_manager()->GenerateToken( + rpc->user_credentials().real_user(), + &authn_token); + if (!s.ok()) { + KLOG_EVERY_N_SECS(WARNING, 1) + << "Unable to generate signed token for " << rpc->requestor_string() + << ": " << s.ToString(); + } else { + // TODO(todd): this might be a good spot for some auditing code? + resp->mutable_authn_token()->Swap(&authn_token); + } + } rpc->RespondSuccess(); } http://git-wip-us.apache.org/repos/asf/kudu/blob/7d434601/src/kudu/rpc/outbound_call.h ---------------------------------------------------------------------- diff --git a/src/kudu/rpc/outbound_call.h b/src/kudu/rpc/outbound_call.h index bd5163e..7df4a31 100644 --- a/src/kudu/rpc/outbound_call.h +++ b/src/kudu/rpc/outbound_call.h @@ -57,6 +57,7 @@ class RpcController; // // TODO(mpercy): this is actually used server side too -- should // we instead introduce a RemoteUser class or something? +// TODO(todd): this should move into a standalone header. class UserCredentials { public: UserCredentials(); http://git-wip-us.apache.org/repos/asf/kudu/blob/7d434601/src/kudu/security/token.proto ---------------------------------------------------------------------- diff --git a/src/kudu/security/token.proto b/src/kudu/security/token.proto index 79e5490..26284be 100644 --- a/src/kudu/security/token.proto +++ b/src/kudu/security/token.proto @@ -18,7 +18,10 @@ package kudu.security; option java_package = "org.apache.kudu.security"; +import "kudu/util/pb_util.proto"; + message AuthnTokenPB { + optional string username = 1; }; message AuthzTokenPB { @@ -63,7 +66,7 @@ message SignedTokenPB { optional bytes token_data = 1; // The cryptographic signature of 'token_contents'. - optional bytes signature = 2; + optional bytes signature = 2 [ (kudu.REDACT) = true ]; // The sequence number of the key which produced 'signature'. optional int64 signing_key_seq_num = 3;