You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by to...@apache.org on 2017/01/24 17:19:07 UTC

[2/3] kudu git commit: ca: allow creating a self-signed CA

ca: allow creating a self-signed CA

This adds the ability to initialize a CertSigner in a self-signed mode.
In this mode, the issuer on the generated cert is set to the subject in
the CSR.

This required a bit of reworking of the initialization sequence for
CertSigner. Now instead of exclusively owning a Key/Cert, it uses
shared_ptrs to share ownership.

Along the way, also changed the Init() methods to enforce that they're
only called once. This invalidated one of the existing test cases, but
is simpler to understand, and there weren't any real use cases to do
initialization from multiple threads.

Change-Id: If8bfa3cc014f1c671ced549a8297ba46065e1124
Reviewed-on: http://gerrit.cloudera.org:8080/5765
Tested-by: Kudu Jenkins
Reviewed-by: Alexey Serbin <as...@cloudera.com>
Reviewed-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/3ef4fba2
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/3ef4fba2
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/3ef4fba2

Branch: refs/heads/master
Commit: 3ef4fba27df094ec6e119a0176fd46cde13e5ff5
Parents: a1a778c
Author: Todd Lipcon <to...@apache.org>
Authored: Sun Jan 22 17:50:50 2017 -0800
Committer: Todd Lipcon <to...@apache.org>
Committed: Tue Jan 24 05:44:41 2017 +0000

----------------------------------------------------------------------
 src/kudu/security/ca/cert_management.cc        |  82 +++++++----
 src/kudu/security/ca/cert_management.h         |  51 +++++--
 src/kudu/security/test/cert_management-test.cc | 143 ++++++++++----------
 3 files changed, 167 insertions(+), 109 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/3ef4fba2/src/kudu/security/ca/cert_management.cc
----------------------------------------------------------------------
diff --git a/src/kudu/security/ca/cert_management.cc b/src/kudu/security/ca/cert_management.cc
index e34a7b4..f699a9e 100644
--- a/src/kudu/security/ca/cert_management.cc
+++ b/src/kudu/security/ca/cert_management.cc
@@ -43,6 +43,7 @@
 using std::lock_guard;
 using std::move;
 using std::ostringstream;
+using std::shared_ptr;
 using std::string;
 using strings::Substitute;
 
@@ -341,9 +342,7 @@ Status CertRequestGenerator::Init() {
   InitializeOpenSSL();
 
   lock_guard<simple_spinlock> guard(lock_);
-  if (is_initialized_) {
-    return Status::OK();
-  }
+  CHECK(!is_initialized_);
   if (config_.uuid.empty()) {
     return Status::InvalidArgument("missing end-entity UUID/name");
   }
@@ -482,26 +481,55 @@ Status CaCertRequestGenerator::SetExtensions(X509_REQ* req) const {
   return Status::OK();
 }
 
-CertSigner::CertSigner(Config config)
-    : config_(move(config)),
-      is_initialized_(false) {
+CertSigner::CertSigner()
+    : is_initialized_(false) {
 }
 
-Status CertSigner::Init() {
+Status CertSigner::Init(shared_ptr<Cert> ca_cert,
+                        shared_ptr<Key> ca_private_key) {
+  CHECK(ca_cert && ca_cert->GetRawData());
+  CHECK(ca_private_key && ca_private_key->GetRawData());
   InitializeOpenSSL();
 
-  lock_guard<simple_spinlock> guard(lock_);
-  if (is_initialized_) {
-    return Status::OK();
-  }
-  RETURN_NOT_OK(ca_cert_.FromFile(config_.ca_cert_path, DataFormat::PEM));
-  RETURN_NOT_OK(ca_private_key_.FromFile(config_.ca_private_key_path,
-                DataFormat::PEM));
-  CERT_RET_NOT_OK(X509_check_private_key(ca_cert_.GetRawData(),
-                                         ca_private_key_.GetRawData()),
+  std::lock_guard<simple_spinlock> guard(lock_);
+  CHECK(!is_initialized_);
+
+  ca_cert_ = std::move(ca_cert);
+  ca_private_key_ = std::move(ca_private_key);
+  is_initialized_ = true;
+  return Status::OK();
+}
+
+Status CertSigner::InitForSelfSigning(shared_ptr<Key> private_key) {
+  CHECK(private_key);
+  InitializeOpenSSL();
+
+  std::lock_guard<simple_spinlock> guard(lock_);
+  CHECK(!is_initialized_);
+
+  ca_private_key_ = std::move(private_key);
+  is_initialized_ = true;
+  return Status::OK();
+}
+
+Status CertSigner::InitFromFiles(const string& ca_cert_path,
+                                 const string& ca_private_key_path) {
+  InitializeOpenSSL();
+
+  std::lock_guard<simple_spinlock> guard(lock_);
+  CHECK(!is_initialized_);
+  auto cert = std::make_shared<Cert>();
+  auto key = std::make_shared<Key>();
+  RETURN_NOT_OK(cert->FromFile(ca_cert_path, DataFormat::PEM));
+  RETURN_NOT_OK(key->FromFile(ca_private_key_path,
+                              DataFormat::PEM));
+  CERT_RET_NOT_OK(X509_check_private_key(cert->GetRawData(),
+                                         key->GetRawData()),
                   Substitute("$0, $1: CA certificate and private key "
                              "do not match",
-                             config_.ca_cert_path, config_.ca_private_key_path));
+                             ca_cert_path, ca_private_key_path));
+  ca_cert_ = std::move(cert);
+  ca_private_key_ = std::move(key);
   is_initialized_ = true;
   return Status::OK();
 }
@@ -514,13 +542,13 @@ bool CertSigner::Initialized() const {
 const Cert& CertSigner::ca_cert() const {
   lock_guard<simple_spinlock> guard(lock_);
   DCHECK(is_initialized_);
-  return ca_cert_;
+  return *CHECK_NOTNULL(ca_cert_.get());
 }
 
 const Key& CertSigner::ca_private_key() const {
   lock_guard<simple_spinlock> guard(lock_);
   DCHECK(is_initialized_);
-  return ca_private_key_;
+  return *ca_private_key_;
 }
 
 Status CertSigner::Sign(const CertSignRequest& req, Cert* ret) const {
@@ -528,7 +556,7 @@ Status CertSigner::Sign(const CertSignRequest& req, Cert* ret) const {
   CHECK(Initialized());
   auto x509 = make_ssl_unique(X509_new());
   RETURN_NOT_OK(FillCertTemplateFromRequest(req.GetRawData(), x509.get()));
-  RETURN_NOT_OK(DoSign(EVP_sha256(), config_.exp_interval_sec, x509.get()));
+  RETURN_NOT_OK(DoSign(EVP_sha256(), exp_interval_sec_, x509.get()));
   ret->AdoptRawData(x509.release());
 
   return Status::OK();
@@ -608,14 +636,18 @@ Status CertSigner::GenerateSerial(c_unique_ptr<ASN1_INTEGER>* ret) {
 
 Status CertSigner::DoSign(const EVP_MD* digest, int32_t exp_seconds,
                           X509* ret) const {
+  CHECK(ret);
+
   // Version 3 (v3) of X509 certificates. The integer value is one less
   // than the version it represents. This is not a typo. :)
   static const int kX509V3 = 2;
 
-  CERT_RET_NOT_OK(
-      X509_set_issuer_name(CHECK_NOTNULL(ret),
-                           X509_get_subject_name(ca_cert_.GetRawData())),
-      "error setting issuer name");
+  // If we have a CA cert, then the CA is the issuer.
+  // Otherwise, we are self-signing so the target cert is also the issuer.
+  X509* issuer_cert = ca_cert_ ? ca_cert_->GetRawData() : ret;
+  X509_NAME* issuer_name = X509_get_subject_name(issuer_cert);
+  CERT_RET_NOT_OK(X509_set_issuer_name(ret, issuer_name),
+                  "error setting issuer name");
   c_unique_ptr<ASN1_INTEGER> serial;
   RETURN_NOT_OK(GenerateSerial(&serial));
   // set version to v3
@@ -626,7 +658,7 @@ Status CertSigner::DoSign(const EVP_MD* digest, int32_t exp_seconds,
                    "error setting cert validity time");
   CERT_RET_IF_NULL(X509_gmtime_adj(X509_get_notAfter(ret), exp_seconds),
                    "error setting cert expiration time");
-  RETURN_NOT_OK(DigestSign(digest, ca_private_key_.GetRawData(), ret));
+  RETURN_NOT_OK(DigestSign(digest, ca_private_key_->GetRawData(), ret));
 
   return Status::OK();
 }

http://git-wip-us.apache.org/repos/asf/kudu/blob/3ef4fba2/src/kudu/security/ca/cert_management.h
----------------------------------------------------------------------
diff --git a/src/kudu/security/ca/cert_management.h b/src/kudu/security/ca/cert_management.h
index 65faafd..2d623d8 100644
--- a/src/kudu/security/ca/cert_management.h
+++ b/src/kudu/security/ca/cert_management.h
@@ -20,12 +20,14 @@
 #include <functional>
 #include <map>
 #include <memory>
+#include <mutex>
 #include <string>
 #include <vector>
 
 #include "kudu/gutil/macros.h"
 #include "kudu/security/openssl_util.h"
 #include "kudu/util/locks.h"
+#include "kudu/util/monotime.h"
 #include "kudu/util/status.h"
 
 // Forward declarations for the relevant OpenSSL typedefs
@@ -215,16 +217,34 @@ class CaCertRequestGenerator : public CertRequestGeneratorBase {
 // An utility class for issuing and signing certificates.
 class CertSigner {
  public:
-  struct Config {
-    const int32_t exp_interval_sec;
-    const std::string ca_cert_path;
-    const std::string ca_private_key_path;
-  };
-
-  explicit CertSigner(Config config);
+  // Create a CertSigner.
+  // Exactly one of the following Init*() methods must be called
+  // exactly once before the instance may be used.
+  CertSigner();
   ~CertSigner() = default;
 
-  Status Init();
+  // Initialize the signer from existing Cert/Key objects.
+  // The passed objects must be initialized.
+  Status Init(std::shared_ptr<Cert> ca_cert,
+              std::shared_ptr<Key> ca_private_key);
+
+  // Initialize the signer from a CA cert and private key stored
+  // on disk.
+  Status InitFromFiles(const std::string& ca_cert_path,
+                       const std::string& ca_private_key_path);
+  // Initialize the signer for self-signing using the given private key.
+  //
+  // Any certificates signed by this CertSigner will have the 'issuer' equal
+  // to the signed cert's subject.
+  Status InitForSelfSigning(std::shared_ptr<Key> private_key);
+
+  // Set the expiration interval for certs signed by this signer.
+  // This may be changed at any point.
+  void set_expiration_interval(MonoDelta expiration) {
+    std::lock_guard<simple_spinlock> l(lock_);
+    exp_interval_sec_ = expiration.ToSeconds();
+  }
+
   bool Initialized() const;
 
   const Cert& ca_cert() const;
@@ -240,13 +260,18 @@ class CertSigner {
 
   Status DoSign(const EVP_MD* digest, int32_t exp_seconds, X509 *ret) const;
 
-  const std::string ca_cert_path_;
-  const std::string ca_private_key_path_;
-  const Config config_;
   mutable simple_spinlock lock_;
   bool is_initialized_; // protected by lock_
-  Cert ca_cert_;
-  Key ca_private_key_;
+
+  // The expiration interval of certs signed by this signer.
+  int32_t exp_interval_sec_ = 24 * 60 * 60;
+
+  // The CA cert. null if this CertSigner is configured for self-signing.
+  std::shared_ptr<Cert> ca_cert_;
+
+  // The CA private key. If configured for self-signing, this is the
+  // private key associated with the target cert.
+  std::shared_ptr<Key> ca_private_key_;
 
   DISALLOW_COPY_AND_ASSIGN(CertSigner);
 };

http://git-wip-us.apache.org/repos/asf/kudu/blob/3ef4fba2/src/kudu/security/test/cert_management-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/security/test/cert_management-test.cc b/src/kudu/security/test/cert_management-test.cc
index 0273248..857ba9b 100644
--- a/src/kudu/security/test/cert_management-test.cc
+++ b/src/kudu/security/test/cert_management-test.cc
@@ -112,14 +112,9 @@ class CertManagementTest : public KuduTest {
     if (SINGLE_INIT == init_type) {
       ASSERT_OK(gen_shared.Init());
     }
-    const CertSigner::Config signer_config = {
-      .exp_interval_sec = 24 * 60 * 60,
-      .ca_cert_path = ca_cert_file_,
-      .ca_private_key_path = ca_private_key_file_,
-    };
-    CertSigner signer_shared(signer_config);
+    CertSigner signer_shared;
     if (SINGLE_INIT == init_type) {
-      ASSERT_OK(signer_shared.Init());
+      ASSERT_OK(signer_shared.InitFromFiles(ca_cert_file_, ca_private_key_file_));
     }
 
     vector<thread> threads;
@@ -129,14 +124,14 @@ class CertManagementTest : public KuduTest {
       threads.emplace_back([&, thread_idx]() {
         for (size_t i = 0; i < iter_num; ++i) {
           CertRequestGenerator gen_local(gen_config);
-          CertSigner signer_local(signer_config);
+          CertSigner signer_local;
 
           CertRequestGenerator& gen = (SHARED == sharing_type) ? gen_shared
                                                                : gen_local;
           CertSigner& signer = (SHARED == sharing_type) ? signer_shared
                                                         : signer_local;
 
-          if (DEDICATED == sharing_type || MULTIPLE_INIT == init_type) {
+          if (DEDICATED == sharing_type) {
             CHECK_OK(gen.Init());
           }
           const size_t sel = i % 4;
@@ -145,8 +140,8 @@ class CertManagementTest : public KuduTest {
           CHECK_OK(GeneratePrivateKey(key_bits, &key));
           CertSignRequest req;
           CHECK_OK(gen.GenerateRequest(key, &req));
-          if (DEDICATED == sharing_type || MULTIPLE_INIT == init_type) {
-            CHECK_OK(signer.Init());
+          if (DEDICATED == sharing_type) {
+            CHECK_OK(signer.InitFromFiles(ca_cert_file_, ca_private_key_file_));
           }
           Cert cert;
           CHECK_OK(signer.Sign(req, &cert));
@@ -266,38 +261,23 @@ TEST_F(CertManagementTest, SignerInitWithWrongFiles) {
   // Providing files which guaranteed to exists, but do not contain valid data.
   // This is to make sure the init handles that situation correctly and
   // does not choke on the wrong input data.
-  const CertSigner::Config config = {
-    .exp_interval_sec = 24 * 60 * 60,
-    .ca_cert_path = "/bin/sh",
-    .ca_private_key_path = "/bin/cat",
-  };
-  CertSigner signer(config);
-  ASSERT_FALSE(signer.Init().ok());
+  CertSigner signer;
+  ASSERT_FALSE(signer.InitFromFiles("/bin/sh", "/bin/cat").ok());
 }
 
 // Check that CertSigner behaves in a predictable way if given non-matching
 // CA private key and certificate.
 TEST_F(CertManagementTest, SignerInitWithMismatchedCertAndKey) {
   {
-    const CertSigner::Config config = {
-      .exp_interval_sec = 24 * 60 * 60,
-      .ca_cert_path = ca_cert_file_,
-      .ca_private_key_path = ca_exp_private_key_file_,
-    };
-    CertSigner signer(config);
-    Status s = signer.Init();
+    CertSigner signer;
+    Status s = signer.InitFromFiles(ca_cert_file_, ca_exp_private_key_file_);
     const string err_msg = s.ToString();
     ASSERT_TRUE(s.IsRuntimeError()) << err_msg;
     ASSERT_STR_CONTAINS(err_msg, "CA certificate and private key do not match");
   }
   {
-    const CertSigner::Config config = {
-      .exp_interval_sec = 24 * 60 * 60,
-      .ca_cert_path = ca_exp_cert_file_,
-      .ca_private_key_path = ca_private_key_file_,
-    };
-    CertSigner signer(config);
-    Status s = signer.Init();
+    CertSigner signer;
+    Status s = signer.InitFromFiles(ca_exp_cert_file_, ca_private_key_file_);
     const string err_msg = s.ToString();
     ASSERT_TRUE(s.IsRuntimeError()) << err_msg;
     ASSERT_STR_CONTAINS(err_msg, "CA certificate and private key do not match");
@@ -315,14 +295,9 @@ TEST_F(CertManagementTest, SignerInitWithExpiredCert) {
   ASSERT_OK(gen.Init());
   CertSignRequest req;
   ASSERT_OK(gen.GenerateRequest(key, &req));
-  const CertSigner::Config signer_config = {
-    .exp_interval_sec = 24 * 60 * 60,
-    .ca_cert_path = ca_exp_cert_file_,
-    .ca_private_key_path = ca_exp_private_key_file_,
-  };
-  CertSigner signer(signer_config);
+  CertSigner signer;
   // Even if the certificate is expired, the signer should initialize OK.
-  ASSERT_OK(signer.Init());
+  ASSERT_OK(signer.InitFromFiles(ca_exp_cert_file_, ca_exp_private_key_file_));
   Cert cert;
   // Signer works fine even with expired CA certificate.
   ASSERT_OK(signer.Sign(req, &cert));
@@ -340,13 +315,8 @@ TEST_F(CertManagementTest, SignCert) {
   ASSERT_OK(gen.Init());
   CertSignRequest req;
   ASSERT_OK(gen.GenerateRequest(key, &req));
-  const CertSigner::Config signer_config = {
-    .exp_interval_sec = 24 * 60 * 60,
-    .ca_cert_path = ca_cert_file_,
-    .ca_private_key_path = ca_private_key_file_,
-  };
-  CertSigner signer(signer_config);
-  ASSERT_OK(signer.Init());
+  CertSigner signer;
+  ASSERT_OK(signer.InitFromFiles(ca_cert_file_, ca_private_key_file_));
   Cert cert;
   ASSERT_OK(signer.Sign(req, &cert));
 }
@@ -361,17 +331,60 @@ TEST_F(CertManagementTest, SignCaCert) {
   ASSERT_OK(gen.Init());
   CertSignRequest req;
   ASSERT_OK(gen.GenerateRequest(key, &req));
-  const CertSigner::Config signer_config = {
-    .exp_interval_sec = 24 * 60 * 60,
-    .ca_cert_path = ca_cert_file_,
-    .ca_private_key_path = ca_private_key_file_,
-  };
-  CertSigner signer(signer_config);
-  ASSERT_OK(signer.Init());
+  CertSigner signer;
+  ASSERT_OK(signer.InitFromFiles(ca_cert_file_, ca_private_key_file_));
   Cert cert;
   ASSERT_OK(signer.Sign(req, &cert));
 }
 
+// Test the creation and use of a CA which uses a self-signed CA cert
+// generated on the fly.
+TEST_F(CertManagementTest, TestSelfSignedCA) {
+  // Create a key for the self-signed CA.
+  auto ca_key = std::make_shared<Key>();
+  ASSERT_OK(GeneratePrivateKey(2048, ca_key.get()));
+
+  // Generate a CSR for the CA.
+  CertSignRequest ca_csr;
+  {
+    const CertRequestGenerator::Config gen_config(
+        PrepareConfig("8C084CF6-A30B-4F5B-9673-A73E62E29A9D"));
+    CaCertRequestGenerator gen(gen_config);
+    ASSERT_OK(gen.Init());
+    ASSERT_OK(gen.GenerateRequest(*ca_key, &ca_csr));
+  }
+
+  // Self-sign the CA's CSR.
+  auto ca_cert = std::make_shared<Cert>();
+  {
+    CertSigner ca_signer;
+    ASSERT_OK(ca_signer.InitForSelfSigning(ca_key));
+    ASSERT_OK(ca_signer.Sign(ca_csr, ca_cert.get()));
+  }
+
+  // Create a key for the tablet server.
+  auto ts_key = std::make_shared<Key>();
+  ASSERT_OK(GeneratePrivateKey(2048, ts_key.get()));
+
+  // Prepare a CSR for a tablet server that wants signing.
+  CertSignRequest ts_csr;
+  {
+    CertRequestGenerator gen(PrepareConfig(
+        "some-tablet-server",
+        {"localhost"}, {"127.0.0.1", "127.0.10.20"}));
+    ASSERT_OK(gen.Init());
+    ASSERT_OK(gen.GenerateRequest(*ts_key, &ts_csr));
+  }
+
+  // Sign it using the self-signed CA.
+  Cert ts_cert;
+  {
+    CertSigner signer;
+    ASSERT_OK(signer.Init(ca_cert, ca_key));
+    ASSERT_OK(signer.Sign(ts_csr, &ts_cert));
+  }
+}
+
 // Check the transformation chains for keys:
 //   internal -> PEM -> internal -> PEM
 //   internal -> DER -> internal -> DER
@@ -437,13 +450,8 @@ TEST_F(CertManagementTest, X509FromAndToString) {
   CertSignRequest req;
   ASSERT_OK(gen.GenerateRequest(key, &req));
 
-  const CertSigner::Config signer_config = {
-    .exp_interval_sec = 24 * 60 * 60,
-    .ca_cert_path = ca_cert_file_,
-    .ca_private_key_path = ca_private_key_file_,
-  };
-  CertSigner signer(signer_config);
-  ASSERT_OK(signer.Init());
+  CertSigner signer;
+  ASSERT_OK(signer.InitFromFiles(ca_cert_file_, ca_private_key_file_));
   Cert cert_ref;
   ASSERT_OK(signer.Sign(req, &cert_ref));
 
@@ -461,8 +469,8 @@ TEST_F(CertManagementTest, X509FromAndToString) {
 
 // Generate CSR and issue corresponding certificate in a multi-threaded fashion:
 // every thread uses its own instances of CertRequestGenerator and CertSigner,
-// which were initialized eariler (i.e. those threads do not call Init()).
-TEST_F(CertManagementTest, SignMultiThreadExclusiveSignleInit) {
+// which were initialized earlier (i.e. those threads do not call Init()).
+TEST_F(CertManagementTest, SignMultiThreadExclusiveSingleInit) {
   ASSERT_NO_FATAL_FAILURE(SignMultiThread(32, 16, DEDICATED, SINGLE_INIT));
 }
 
@@ -475,18 +483,11 @@ TEST_F(CertManagementTest, SignMultiThreadExclusiveMultiInit) {
 
 // Generate CSR and issue corresponding certificate in a multi-thread fashion:
 // all threads use shared instances of CertRequestGenerator and CertSigner,
-// which were initialized eariler (i.e. those threads do not call Init()).
-TEST_F(CertManagementTest, SignMultiThreadSharedSignleInit) {
+// which were initialized earlier (i.e. those threads do not call Init()).
+TEST_F(CertManagementTest, SignMultiThreadSharedSingleInit) {
   ASSERT_NO_FATAL_FAILURE(SignMultiThread(32, 16, SHARED, SINGLE_INIT));
 }
 
-// Generate CSR and issue corresponding certificate in a multi-thread fashion:
-// all threads use shared instances of CertRequestGenerator and CertSigner,
-// where every threads initializes those objects by itself.
-TEST_F(CertManagementTest, SignMultiThreadSharedMultiInit) {
-  ASSERT_NO_FATAL_FAILURE(SignMultiThread(16, 32, SHARED, MULTIPLE_INIT));
-}
-
 } // namespace ca
 } // namespace security
 } // namespace kudu