Repository: kudu
Updated Branches:
refs/heads/master 4a0fa0921 -> ae5ed629a
IPKI: master CA should verify x509 attributes before signing certs
This makes the master verify that the attributes stored in the X509 CSR
match the authenticated user's credentials.
Change-Id: I390e113220302dec335afc38d05d2ea73c965ba6
Reviewed-on: http://gerrit.cloudera.org:8080/6118
Tested-by: Kudu Jenkins
Reviewed-by: Alexey Serbin <aserbin@cloudera.com>
Project: http://git-wip-us.apache.org/repos/asf/kudu/repo
Commit: http://git-wip-us.apache.org/repos/asf/kudu/commit/362eb53e
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/362eb53e
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/362eb53e
Branch: refs/heads/master
Commit: 362eb53efffe05c13fdff824ea4656a458b79230
Parents: 4a0fa09
Author: Todd Lipcon <todd@apache.org>
Authored: Wed Feb 22 15:20:01 2017 -0800
Committer: Todd Lipcon <todd@apache.org>
Committed: Thu Feb 23 04:00:04 2017 +0000
----------------------------------------------------------------------
.../master_cert_authority-itest.cc | 90 ++++++++++++++------
src/kudu/master/master_cert_authority.cc | 29 +++++--
src/kudu/master/master_cert_authority.h | 9 +-
src/kudu/master/master_service.cc | 3 +-
src/kudu/rpc/negotiation-test.cc | 4 +-
src/kudu/rpc/remote_user.cc | 2 +-
src/kudu/rpc/remote_user.h | 9 +-
7 files changed, 100 insertions(+), 46 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/kudu/blob/362eb53e/src/kudu/integration-tests/master_cert_authority-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/master_cert_authority-itest.cc b/src/kudu/integration-tests/master_cert_authority-itest.cc
index 227c4cd..c6873de 100644
--- a/src/kudu/integration-tests/master_cert_authority-itest.cc
+++ b/src/kudu/integration-tests/master_cert_authority-itest.cc
@@ -34,6 +34,7 @@
#include "kudu/security/crypto.h"
#include "kudu/util/pb_util.h"
#include "kudu/util/test_util.h"
+#include "kudu/util/user.h"
using std::string;
using std::shared_ptr;
@@ -63,6 +64,9 @@ class MasterCertAuthorityTest : public KuduTest {
KuduTest::SetUp();
cluster_.reset(new MiniCluster(env_, opts_));
ASSERT_OK(cluster_->Start());
+
+ rpc::MessengerBuilder bld("Client");
+ ASSERT_OK(bld.Build(&messenger_));
}
Status RestartCluster() {
@@ -90,7 +94,7 @@ class MasterCertAuthorityTest : public KuduTest {
ASSERT_OK(ca_cert->ToString(ca_cert_str, DataFormat::PEM));
}
- void SendRegistrationHBs(const shared_ptr<rpc::Messenger>& messenger) {
+ void SendRegistrationHBs() {
TSToMasterCommonPB common;
common.mutable_ts_instance()->set_permanent_uuid(kFakeTsUUID);
common.mutable_ts_instance()->set_instance_seqno(1);
@@ -114,7 +118,7 @@ class MasterCertAuthorityTest : public KuduTest {
if (!m->is_running()) {
continue;
}
- MasterServiceProxy proxy(messenger, m->bound_rpc_addr());
+ MasterServiceProxy proxy(messenger_, m->bound_rpc_addr());
// All masters (including followers) should accept the heartbeat.
ASSERT_OK(proxy.TSHeartbeat(req, &resp, &rpc));
@@ -123,10 +127,9 @@ class MasterCertAuthorityTest : public KuduTest {
}
}
- void SendCertSignRequestHBs(const shared_ptr<rpc::Messenger>& messenger,
- const string& csr_str,
- bool* has_signed_certificate,
- string* signed_certificate) {
+ Status SendCertSignRequestHBs(const string& csr_str,
+ bool* has_signed_certificate,
+ string* signed_certificate) {
TSToMasterCommonPB common;
common.mutable_ts_instance()->set_permanent_uuid(kFakeTsUUID);
common.mutable_ts_instance()->set_instance_seqno(1);
@@ -145,26 +148,30 @@ class MasterCertAuthorityTest : public KuduTest {
if (!m->is_running()) {
continue;
}
- MasterServiceProxy proxy(messenger, m->bound_rpc_addr());
+ MasterServiceProxy proxy(messenger_, m->bound_rpc_addr());
// All masters (including followers) should accept the heartbeat.
- ASSERT_OK(proxy.TSHeartbeat(req, &resp, &rpc));
+ RETURN_NOT_OK(proxy.TSHeartbeat(req, &resp, &rpc));
SCOPED_TRACE(SecureDebugString(resp));
- ASSERT_FALSE(resp.has_error());
+ if (resp.has_error()) {
+ return Status::RuntimeError("RPC error", resp.error().ShortDebugString());
+ }
// Only the leader sends back the signed server certificate.
if (resp.leader_master()) {
has_leader_master_response = true;
- ASSERT_TRUE(resp.has_signed_cert_der());
ts_cert_str = resp.signed_cert_der();
} else {
- ASSERT_FALSE(resp.has_signed_cert_der());
+ if (resp.has_signed_cert_der()) {
+ return Status::RuntimeError("unexpected cert returned from non-leader");
+ }
}
}
if (has_leader_master_response) {
- *has_signed_certificate = true;
+ *has_signed_certificate = !ts_cert_str.empty();
*signed_certificate = ts_cert_str;
}
+ return Status::OK();
}
protected:
@@ -173,6 +180,8 @@ class MasterCertAuthorityTest : public KuduTest {
int num_masters_;
MiniClusterOptions opts_;
gscoped_ptr<MiniCluster> cluster_;
+
+ shared_ptr<rpc::Messenger> messenger_;
};
const char MasterCertAuthorityTest::kFakeTsUUID[] = "fake-ts-uuid";
@@ -215,25 +224,52 @@ TEST_F(MasterCertAuthorityTest, CertAuthorityOnLeaderRoleSwitch) {
EXPECT_EQ(ref_cert_str, new_leader_cert_str);
}
+
+void GenerateCSR(const CertRequestGenerator::Config& gen_config,
+ string* csr_str) {
+ PrivateKey key;
+ ASSERT_OK(security::GeneratePrivateKey(512, &key));
+ CertRequestGenerator gen(gen_config);
+ ASSERT_OK(gen.Init());
+ CertSignRequest csr;
+ ASSERT_OK(gen.GenerateRequest(key, &csr));
+ ASSERT_OK(csr.ToString(csr_str, DataFormat::DER));
+}
+
+TEST_F(MasterCertAuthorityTest, RefuseToSignInvalidCSR) {
+ NO_FATALS(SendRegistrationHBs());
+ string csr_str;
+ {
+ CertRequestGenerator::Config gen_config;
+ gen_config.cn = "ts.foo.com";
+ gen_config.user_id = "joe-impostor";
+ NO_FATALS(GenerateCSR(gen_config, &csr_str));
+ }
+ ASSERT_OK(WaitForLeader());
+ {
+ string ts_cert_str;
+ bool has_ts_cert = false;
+ Status s = SendCertSignRequestHBs(csr_str, &has_ts_cert, &ts_cert_str);
+ ASSERT_STR_MATCHES(s.ToString(),
+ "Remote error: Not authorized: invalid CSR: CSR did not "
+ "contain expected username. "
+ "\\(CSR: 'joe-impostor' RPC: '.*'\\)");
+ }
+}
+
// Test that every master accepts heartbeats, but only the leader master
// responds with signed certificate if a heartbeat contains the CSR field.
TEST_F(MasterCertAuthorityTest, MasterLeaderSignsCSR) {
- shared_ptr<rpc::Messenger> messenger;
- rpc::MessengerBuilder bld("Client");
- ASSERT_OK(bld.Build(&messenger));
- NO_FATALS(SendRegistrationHBs(messenger));
+ NO_FATALS(SendRegistrationHBs());
string csr_str;
{
CertRequestGenerator::Config gen_config;
gen_config.cn = "ts.foo.com";
- PrivateKey key;
- ASSERT_OK(security::GeneratePrivateKey(512, &key));
- CertRequestGenerator gen(gen_config);
- ASSERT_OK(gen.Init());
- CertSignRequest csr;
- ASSERT_OK(gen.GenerateRequest(key, &csr));
- ASSERT_OK(csr.ToString(&csr_str, DataFormat::DER));
+ string test_user;
+ ASSERT_OK(GetLoggedInUser(&test_user));
+ gen_config.user_id = test_user;
+ NO_FATALS(GenerateCSR(gen_config, &csr_str));
}
// Make sure a tablet server receives signed certificate from
@@ -242,8 +278,7 @@ TEST_F(MasterCertAuthorityTest, MasterLeaderSignsCSR) {
{
string ts_cert_str;
bool has_ts_cert = false;
- NO_FATALS(SendCertSignRequestHBs(messenger, csr_str,
- &has_ts_cert, &ts_cert_str));
+ NO_FATALS(SendCertSignRequestHBs(csr_str, &has_ts_cert, &ts_cert_str));
ASSERT_TRUE(has_ts_cert);
// Try to load the certificate to check that the data is not corrupted.
@@ -260,12 +295,11 @@ TEST_F(MasterCertAuthorityTest, MasterLeaderSignsCSR) {
// Re-register with the new leader.
ASSERT_OK(WaitForLeader());
- NO_FATALS(SendRegistrationHBs(messenger));
+ NO_FATALS(SendRegistrationHBs());
{
string ts_cert_str;
bool has_ts_cert = false;
- NO_FATALS(SendCertSignRequestHBs(messenger, csr_str,
- &has_ts_cert, &ts_cert_str));
+ NO_FATALS(SendCertSignRequestHBs(csr_str, &has_ts_cert, &ts_cert_str));
ASSERT_TRUE(has_ts_cert);
// Try to load the certificate to check that the data is not corrupted.
Cert ts_cert;
http://git-wip-us.apache.org/repos/asf/kudu/blob/362eb53e/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 f5490f9..2b3a31d 100644
--- a/src/kudu/master/master_cert_authority.cc
+++ b/src/kudu/master/master_cert_authority.cc
@@ -21,8 +21,10 @@
#include <string>
#include <utility>
+#include <boost/optional.hpp>
#include <gflags/gflags.h>
+#include "kudu/rpc/remote_user.h"
#include "kudu/security/ca/cert_management.h"
#include "kudu/security/cert.h"
#include "kudu/security/crypto.h"
@@ -30,6 +32,7 @@
#include "kudu/util/flag_tags.h"
#include "kudu/util/status.h"
+using boost::optional;
using std::string;
using std::unique_ptr;
@@ -105,20 +108,34 @@ Status MasterCertAuthority::SignServerCSR(const CertSignRequest&
csr,
return Status::OK();
}
-Status MasterCertAuthority::SignServerCSR(const string& csr_der, string* cert_der) {
+Status MasterCertAuthority::SignServerCSR(const string& csr_der, const rpc::RemoteUser&
user,
+ string* cert_der) {
CHECK(ca_cert_ && ca_private_key_) << "not initialized";
- // TODO(PKI): before signing, should we somehow verify the CSR's
- // hostname/server_uuid matches what we think is the hostname? can the signer
- // modify the CSR to add fields, etc, indicating when/where it was signed?
- // maybe useful for debugging.
-
CertSignRequest csr;
RETURN_NOT_OK_PREPEND(csr.FromString(csr_der, security::DataFormat::DER),
"could not parse CSR");
Cert cert;
RETURN_NOT_OK(SignServerCSR(csr, &cert));
+ // Validate that the cert has an included user ID.
+ // It may seem funny to validate after signing, but we already have the functions
+ // to get the cert details out of a Cert object, and not out of a CSR object.
+ optional<string> cert_uid = cert.UserId();
+ if (cert_uid != user.username()) {
+ return Status::NotAuthorized(strings::Substitute(
+ "CSR did not contain expected username. (CSR: '$0' RPC: '$1')",
+ cert_uid.value_or(""),
+ user.username()));
+ }
+ optional<string> cert_principal = cert.KuduKerberosPrincipal();
+ if (cert_principal != user.principal()) {
+ return Status::NotAuthorized(strings::Substitute(
+ "CSR did not contain expected krb5 principal (CSR: '$0' RPC: '$1')",
+ cert_principal.value_or(""),
+ user.principal().value_or("")));
+ }
+
RETURN_NOT_OK_PREPEND(cert.ToString(cert_der, security::DataFormat::DER),
"failed to signed cert as DER format");
return Status::OK();
http://git-wip-us.apache.org/repos/asf/kudu/blob/362eb53e/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 fd7d675..8ca0297 100644
--- a/src/kudu/master/master_cert_authority.h
+++ b/src/kudu/master/master_cert_authority.h
@@ -39,6 +39,10 @@ class CertSigner;
} // namespace ca
} // namespace security
+namespace rpc {
+class RemoteUser;
+} // namespace rpc
+
namespace master {
class MasterCertAuthorityTest;
@@ -70,6 +74,8 @@ class MasterCertAuthority {
std::unique_ptr<security::Cert> cert);
// Sign the given CSR 'csr_der' provided by a server in the cluster.
+ // The authenticated user should be passed in 'caller'. The cert contents
+ // are verified to match the authenticated user.
//
// The CSR should be provided in the DER format.
// The resulting certificate, also in DER format, is returned in 'cert_der'.
@@ -79,7 +85,8 @@ class MasterCertAuthority {
// NOTE: This method is not going to be called in parallel with Init()
// due to the current design, so there is no internal synchronization
// to keep the internal state consistent.
- Status SignServerCSR(const std::string& csr_der, std::string* cert_der);
+ Status SignServerCSR(const std::string& csr_der, const rpc::RemoteUser& caller,
+ std::string* cert_der);
// Same as above, but with objects instead of the DER format CSR/cert.
Status SignServerCSR(const security::CertSignRequest& csr, security::Cert* cert);
http://git-wip-us.apache.org/repos/asf/kudu/blob/362eb53e/src/kudu/master/master_service.cc
----------------------------------------------------------------------
diff --git a/src/kudu/master/master_service.cc b/src/kudu/master/master_service.cc
index 0f3cc19..6bdb518 100644
--- a/src/kudu/master/master_service.cc
+++ b/src/kudu/master/master_service.cc
@@ -155,7 +155,8 @@ void MasterServiceImpl::TSHeartbeat(const TSHeartbeatRequestPB* req,
// 6. Only leaders sign CSR from tablet servers (if present).
if (is_leader_master && req->has_csr_der()) {
string cert;
- Status s = server_->cert_authority()->SignServerCSR(req->csr_der(), &cert);
+ Status s = server_->cert_authority()->SignServerCSR(
+ req->csr_der(), rpc->remote_user(), &cert);
if (!s.ok()) {
rpc->RespondFailure(s.CloneAndPrepend("invalid CSR"));
return;
http://git-wip-us.apache.org/repos/asf/kudu/blob/362eb53e/src/kudu/rpc/negotiation-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/negotiation-test.cc b/src/kudu/rpc/negotiation-test.cc
index d0214eb..b713339 100644
--- a/src/kudu/rpc/negotiation-test.cc
+++ b/src/kudu/rpc/negotiation-test.cc
@@ -301,7 +301,7 @@ TEST_P(TestNegotiation, TestNegotiation) {
break;
case SaslMechanism::GSSAPI:
EXPECT_EQ("client-gssapi", remote_user.username());
- EXPECT_EQ("client-gssapi@KRBTEST.COM", remote_user.principal());
+ EXPECT_EQ("client-gssapi@KRBTEST.COM", remote_user.principal().value_or(""));
break;
case SaslMechanism::INVALID: LOG(FATAL) << "invalid mechanism negotiated";
}
@@ -312,7 +312,7 @@ TEST_P(TestNegotiation, TestNegotiation) {
string expected;
CHECK_OK(GetLoggedInUser(&expected));
EXPECT_EQ(expected, remote_user.username());
- EXPECT_FALSE(remote_user.has_principal());
+ EXPECT_FALSE(remote_user.principal());
break;
}
case AuthenticationType::TOKEN:
http://git-wip-us.apache.org/repos/asf/kudu/blob/362eb53e/src/kudu/rpc/remote_user.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/remote_user.cc b/src/kudu/rpc/remote_user.cc
index 015aace..50e3fcd 100644
--- a/src/kudu/rpc/remote_user.cc
+++ b/src/kudu/rpc/remote_user.cc
@@ -30,7 +30,7 @@ namespace rpc {
string RemoteUser::ToString() const {
string ret;
strings::SubstituteAndAppend(&ret, "{username='$0'", username_);
- if (has_principal()) {
+ if (principal_) {
strings::SubstituteAndAppend(&ret, ", principal='$0'", *principal_);
}
ret.append("}");
http://git-wip-us.apache.org/repos/asf/kudu/blob/362eb53e/src/kudu/rpc/remote_user.h
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/remote_user.h b/src/kudu/rpc/remote_user.h
index 529d2a2..7dc0590 100644
--- a/src/kudu/rpc/remote_user.h
+++ b/src/kudu/rpc/remote_user.h
@@ -19,7 +19,6 @@
#include <string>
#include <boost/optional.hpp>
-#include <glog/logging.h>
namespace kudu {
namespace rpc {
@@ -49,12 +48,8 @@ class RemoteUser {
const std::string& username() const { return username_; }
- bool has_principal() const {
- return principal_ != boost::none;
- }
- const std::string& principal() const {
- DCHECK(has_principal());
- return *principal_;
+ boost::optional<std::string> principal() const {
+ return principal_;
}
void SetAuthenticatedByKerberos(std::string username,
|