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/02/03 17:35:06 UTC

[1/2] kudu git commit: Allow configuring TlsContext with key wrappers

Repository: kudu
Updated Branches:
  refs/heads/master d25430e28 -> 848180654


Allow configuring TlsContext with key wrappers

Change-Id: Idb82427aea5f1bd29bad7529f2d54638b90811e2
Reviewed-on: http://gerrit.cloudera.org:8080/5845
Tested-by: Todd Lipcon <to...@apache.org>
Reviewed-by: Dan Burkert <da...@apache.org>
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/70bbcfc6
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/70bbcfc6
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/70bbcfc6

Branch: refs/heads/master
Commit: 70bbcfc6ad7f117c77f8c24ac0bf544ba1f25dc6
Parents: d25430e
Author: Todd Lipcon <to...@apache.org>
Authored: Tue Jan 31 14:08:56 2017 -0800
Committer: Todd Lipcon <to...@apache.org>
Committed: Fri Feb 3 17:32:07 2017 +0000

----------------------------------------------------------------------
 src/kudu/rpc/messenger.cc                    |  4 +-
 src/kudu/rpc/negotiation-test.cc             |  3 +-
 src/kudu/security/ca/cert_management-test.cc |  8 +-
 src/kudu/security/cert.cc                    | 19 +++++
 src/kudu/security/cert.h                     |  7 ++
 src/kudu/security/tls_context.cc             | 92 ++++++++++++++++++-----
 src/kudu/security/tls_context.h              | 37 +++++++--
 src/kudu/security/tls_handshake-test.cc      | 15 ++--
 8 files changed, 143 insertions(+), 42 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/70bbcfc6/src/kudu/rpc/messenger.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/messenger.cc b/src/kudu/rpc/messenger.cc
index 38c5eaa..da2f364 100644
--- a/src/kudu/rpc/messenger.cc
+++ b/src/kudu/rpc/messenger.cc
@@ -306,8 +306,8 @@ Status Messenger::Init() {
   tls_context_.reset(new security::TlsContext());
   RETURN_NOT_OK(tls_context_->Init());
   if (server_tls_enabled_) {
-    RETURN_NOT_OK(tls_context_->LoadCertificate(FLAGS_rpc_ssl_server_certificate));
-    RETURN_NOT_OK(tls_context_->LoadPrivateKey(FLAGS_rpc_ssl_private_key));
+    RETURN_NOT_OK(tls_context_->LoadCertificateAndKey(FLAGS_rpc_ssl_server_certificate,
+                                                      FLAGS_rpc_ssl_private_key));
     RETURN_NOT_OK(tls_context_->LoadCertificateAuthority(FLAGS_rpc_ssl_certificate_authority));
   }
   for (Reactor* r : reactors_) {

http://git-wip-us.apache.org/repos/asf/kudu/blob/70bbcfc6/src/kudu/rpc/negotiation-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/negotiation-test.cc b/src/kudu/rpc/negotiation-test.cc
index c05386f..7efbe31 100644
--- a/src/kudu/rpc/negotiation-test.cc
+++ b/src/kudu/rpc/negotiation-test.cc
@@ -552,8 +552,7 @@ static void RunTlsGssapiNegotiationServer(unique_ptr<Socket> socket) {
 
   security::TlsContext tls_context;
   ASSERT_OK(tls_context.Init());
-  ASSERT_OK(tls_context.LoadCertificate(server_cert_path));
-  ASSERT_OK(tls_context.LoadPrivateKey(private_key_path));
+  ASSERT_OK(tls_context.LoadCertificateAndKey(server_cert_path, private_key_path));
   server_negotiation.EnableTls(&tls_context);
 
   server_negotiation.set_server_fqdn("127.0.0.1");

http://git-wip-us.apache.org/repos/asf/kudu/blob/70bbcfc6/src/kudu/security/ca/cert_management-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/security/ca/cert_management-test.cc b/src/kudu/security/ca/cert_management-test.cc
index 39dfec8..0a9b3a8 100644
--- a/src/kudu/security/ca/cert_management-test.cc
+++ b/src/kudu/security/ca/cert_management-test.cc
@@ -342,8 +342,7 @@ TEST_F(CertManagementTest, SignerInitWithExpiredCert) {
 // in a single-threaded fashion.
 TEST_F(CertManagementTest, SignCert) {
   const CertRequestGenerator::Config gen_config(
-      PrepareConfig("904A97F9-545A-4746-86D1-85D433FF3F9C",
-                    {"localhost"}, {"127.0.0.1", "127.0.10.20"}));
+      PrepareConfig("test-uuid", {"localhost"}, {"127.0.0.1", "127.0.10.20"}));
   PrivateKey key;
   ASSERT_OK(GeneratePrivateKey(2048, &key));
   CertRequestGenerator gen(gen_config);
@@ -354,6 +353,11 @@ TEST_F(CertManagementTest, SignCert) {
   ASSERT_OK(signer.InitFromFiles(ca_cert_file_, ca_private_key_file_));
   Cert cert;
   ASSERT_OK(signer.Sign(req, &cert));
+
+  EXPECT_EQ("C = US, ST = CA, O = MyCompany, CN = MyName, emailAddress = my@email.com",
+            cert.IssuerName());
+  EXPECT_EQ("C = US, ST = CA, L = San Francisco, O = ASF, OU = The Kudu Project, "
+            "CN = test-uuid", cert.SubjectName());
 }
 
 // Generate X509 CA CSR and sign the result certificate.

http://git-wip-us.apache.org/repos/asf/kudu/blob/70bbcfc6/src/kudu/security/cert.cc
----------------------------------------------------------------------
diff --git a/src/kudu/security/cert.cc b/src/kudu/security/cert.cc
index b141ad1..f04577f 100644
--- a/src/kudu/security/cert.cc
+++ b/src/kudu/security/cert.cc
@@ -46,6 +46,16 @@ template<> struct SslTypeTraits<X509_REQ> {
   static constexpr auto write_der = &i2d_X509_REQ_bio;
 };
 
+string X509NameToString(X509_NAME* name) {
+  CHECK(name);
+  auto bio = ssl_make_unique(BIO_new(BIO_s_mem()));
+  OPENSSL_CHECK_OK(X509_NAME_print_ex(bio.get(), name, 0, XN_FLAG_ONELINE));
+
+  BUF_MEM* membuf;
+  OPENSSL_CHECK_OK(BIO_get_mem_ptr(bio.get(), &membuf));
+  return string(membuf->data, membuf->length);
+}
+
 Status Cert::FromString(const std::string& data, DataFormat format) {
   return ::kudu::security::FromString(data, format, &data_);
 }
@@ -58,6 +68,15 @@ Status Cert::FromFile(const std::string& fpath, DataFormat format) {
   return ::kudu::security::FromFile(fpath, format, &data_);
 }
 
+string Cert::SubjectName() const {
+  return X509NameToString(X509_get_subject_name(data_.get()));
+}
+
+string Cert::IssuerName() const {
+  return X509NameToString(X509_get_issuer_name(data_.get()));
+}
+
+
 Status CertSignRequest::FromString(const std::string& data, DataFormat format) {
   return ::kudu::security::FromString(data, format, &data_);
 }

http://git-wip-us.apache.org/repos/asf/kudu/blob/70bbcfc6/src/kudu/security/cert.h
----------------------------------------------------------------------
diff --git a/src/kudu/security/cert.h b/src/kudu/security/cert.h
index 536d7e9..c21ae0b 100644
--- a/src/kudu/security/cert.h
+++ b/src/kudu/security/cert.h
@@ -24,6 +24,7 @@
 // Forward declarations for the OpenSSL typedefs.
 typedef struct x509_st X509;
 typedef struct X509_req_st X509_REQ;
+typedef struct X509_name_st X509_NAME;
 
 namespace kudu {
 
@@ -31,11 +32,17 @@ class Status;
 
 namespace security {
 
+// Convert an X509_NAME object to a human-readable string.
+std::string X509NameToString(X509_NAME* name);
+
 class Cert : public RawDataWrapper<X509> {
  public:
   Status FromString(const std::string& data, DataFormat format);
   Status ToString(std::string* data, DataFormat format) const;
   Status FromFile(const std::string& fpath, DataFormat format);
+
+  std::string SubjectName() const;
+  std::string IssuerName() const;
 };
 
 class CertSignRequest : public RawDataWrapper<X509_REQ> {

http://git-wip-us.apache.org/repos/asf/kudu/blob/70bbcfc6/src/kudu/security/tls_context.cc
----------------------------------------------------------------------
diff --git a/src/kudu/security/tls_context.cc b/src/kudu/security/tls_context.cc
index 2eec883..b2c4edb 100644
--- a/src/kudu/security/tls_context.cc
+++ b/src/kudu/security/tls_context.cc
@@ -21,8 +21,11 @@
 
 #include <openssl/err.h>
 #include <openssl/ssl.h>
+#include <openssl/x509.h>
 
 #include "kudu/gutil/strings/substitute.h"
+#include "kudu/security/cert.h"
+#include "kudu/security/crypto.h"
 #include "kudu/security/openssl_util.h"
 #include "kudu/security/tls_handshake.h"
 #include "kudu/util/status.h"
@@ -39,8 +42,13 @@ template<> struct SslTypeTraits<SSL> {
 template<> struct SslTypeTraits<SSL_CTX> {
   static constexpr auto free = &SSL_CTX_free;
 };
+template<> struct SslTypeTraits<X509_STORE_CTX> {
+  static constexpr auto free = &X509_STORE_CTX_free;
+};
+
 
-TlsContext::TlsContext() {
+TlsContext::TlsContext()
+    : has_cert_(false) {
   security::InitializeOpenSSL();
 }
 
@@ -75,37 +83,81 @@ Status TlsContext::Init() {
   return Status::OK();
 }
 
-Status TlsContext::LoadCertificate(const string& certificate_path) {
-  ERR_clear_error();
-  errno = 0;
-  if (SSL_CTX_use_certificate_file(ctx_.get(), certificate_path.c_str(), SSL_FILETYPE_PEM) != 1) {
-    return Status::NotFound(Substitute("failed to load certificate file: '$0'", certificate_path),
-                            GetOpenSSLErrors());
+Status TlsContext::VerifyCertChain(const Cert& cert) {
+  X509_STORE* store = SSL_CTX_get_cert_store(ctx_.get());
+  auto store_ctx = ssl_make_unique<X509_STORE_CTX>(X509_STORE_CTX_new());
+
+  OPENSSL_RET_NOT_OK(X509_STORE_CTX_init(store_ctx.get(), store, cert.GetRawData(), nullptr),
+                     "could not init X509_STORE_CTX");
+  int rc = X509_verify_cert(store_ctx.get());
+  if (rc != 1) {
+    int err = X509_STORE_CTX_get_error(store_ctx.get());
+    if (err == X509_V_ERR_DEPTH_ZERO_SELF_SIGNED_CERT) {
+      // It's OK to provide a self-signed cert.
+      return Status::OK();
+    }
+
+    // Get the cert that failed to verify.
+    X509* cur_cert = X509_STORE_CTX_get_current_cert(store_ctx.get());
+    string cert_details;
+    if (cur_cert) {
+      cert_details = Substitute(" (error with cert: subject=$0, issuer=$1)",
+                                X509NameToString(X509_get_subject_name(cur_cert)),
+                                X509NameToString(X509_get_issuer_name(cur_cert)));
+    }
+
+    return Status::RuntimeError(
+        Substitute("could not verify cert chain$0", cert_details),
+        X509_verify_cert_error_string(err));
   }
   return Status::OK();
 }
 
-Status TlsContext::LoadPrivateKey(const string& key_path) {
+Status TlsContext::UseCertificateAndKey(const Cert& cert, const PrivateKey& key) {
   ERR_clear_error();
-  errno = 0;
-  if (SSL_CTX_use_PrivateKey_file(ctx_.get(), key_path.c_str(), SSL_FILETYPE_PEM) != 1) {
-    return Status::NotFound(Substitute("failed to load private key file: '$0'", key_path),
-                            GetOpenSSLErrors());
-  }
+
+  // Verify that the cert and key match.
+  OPENSSL_RET_NOT_OK(X509_check_private_key(cert.GetRawData(), key.GetRawData()),
+                     "cert and private key do not match");
+
+  // Verify that the appropriate CA certs have been loaded into the context
+  // before we adopt a cert. Otherwise, client connections without the CA cert
+  // available would fail.
+  RETURN_NOT_OK(VerifyCertChain(cert));
+
+  OPENSSL_RET_NOT_OK(SSL_CTX_use_PrivateKey(ctx_.get(), key.GetRawData()),
+                     "failed to use private key");
+  OPENSSL_RET_NOT_OK(SSL_CTX_use_certificate(ctx_.get(), cert.GetRawData()),
+                     "failed to use certificate");
+  has_cert_.Store(true);
   return Status::OK();
 }
 
-Status TlsContext::LoadCertificateAuthority(const string& certificate_path) {
+Status TlsContext::AddTrustedCertificate(const Cert& cert) {
+  VLOG(2) << "Trusting certificate " << cert.SubjectName();
+
   ERR_clear_error();
-  errno = 0;
-  if (SSL_CTX_load_verify_locations(ctx_.get(), certificate_path.c_str(), nullptr) != 1) {
-    return Status::NotFound(Substitute("failed to load certificate authority file: '$0'",
-                                       certificate_path),
-                            GetOpenSSLErrors());
-  }
+  auto* cert_store = SSL_CTX_get_cert_store(ctx_.get());
+  OPENSSL_RET_NOT_OK(X509_STORE_add_cert(cert_store, cert.GetRawData()),
+                     "failed to add trusted certificate");
   return Status::OK();
 }
 
+Status TlsContext::LoadCertificateAndKey(const string& certificate_path,
+                                         const string& key_path) {
+  Cert c;
+  RETURN_NOT_OK(c.FromFile(certificate_path, DataFormat::PEM));
+  PrivateKey k;
+  RETURN_NOT_OK(k.FromFile(key_path, DataFormat::PEM));
+  return UseCertificateAndKey(c, k);
+}
+
+Status TlsContext::LoadCertificateAuthority(const string& certificate_path) {
+  Cert c;
+  RETURN_NOT_OK(c.FromFile(certificate_path, DataFormat::PEM));
+  return AddTrustedCertificate(c);
+}
+
 Status TlsContext::InitiateHandshake(TlsHandshakeType handshake_type,
                                      TlsHandshake* handshake) const {
   CHECK(ctx_);

http://git-wip-us.apache.org/repos/asf/kudu/blob/70bbcfc6/src/kudu/security/tls_context.h
----------------------------------------------------------------------
diff --git a/src/kudu/security/tls_context.h b/src/kudu/security/tls_context.h
index 5ba3c0e..0dadfe8 100644
--- a/src/kudu/security/tls_context.h
+++ b/src/kudu/security/tls_context.h
@@ -20,13 +20,16 @@
 #include <functional>
 #include <string>
 
-#include "kudu/security/openssl_util.h"
 #include "kudu/security/tls_handshake.h"
+#include "kudu/util/atomic.h"
 #include "kudu/util/status.h"
 
 namespace kudu {
 namespace security {
 
+class Cert;
+class PrivateKey;
+
 // TlsContext wraps data required by the OpenSSL library for creating TLS
 // protected channels. A single TlsContext instance should be used per server or
 // client instance.
@@ -40,19 +43,41 @@ class TlsContext {
 
   Status Init();
 
-  // Load the server certificate.
-  Status LoadCertificate(const std::string& certificate_path);
+  // Return true if this TlsContext has been configured with a cert and key to
+  // accept TLS connections.
+  bool has_cert() const { return has_cert_.Load(); }
+
+  // Use 'cert' and 'key' as the cert/key for this server/client.
+  //
+  // If the cert is not self-signed, checks that the CA that issued
+  // the signature on 'cert' is already trusted by this context
+  // (e.g. by AddTrustedCertificate).
+  Status UseCertificateAndKey(const Cert& cert, const PrivateKey& key);
+
+  // Add 'cert' as a trusted certificate for this server/client.
+  //
+  // This determines whether other peers are trusted. It also must
+  // be called for any CA certificates that are part of the certificate
+  // chain for the cert passed in 'UseCertificate' above.
+  Status AddTrustedCertificate(const Cert& cert);
 
-  // Load the private key for the server certificate.
-  Status LoadPrivateKey(const std::string& key_path);
+  // Convenience functions for loading cert/CA/key from file paths.
+  // -------------------------------------------------------------
 
-  // Load the certificate authority.
+  // Load the server certificate and key (PEM encoded).
+  Status LoadCertificateAndKey(const std::string& certificate_path,
+                               const std::string& key_path);
+
+  // Load the certificate authority (PEM encoded).
   Status LoadCertificateAuthority(const std::string& certificate_path);
 
   // Initiates a new TlsHandshake instance.
   Status InitiateHandshake(TlsHandshakeType handshake_type, TlsHandshake* handshake) const;
 
  private:
+  Status VerifyCertChain(const Cert& cert);
+
+  AtomicBool has_cert_;
 
   // Owned SSL context.
   c_unique_ptr<SSL_CTX> ctx_;

http://git-wip-us.apache.org/repos/asf/kudu/blob/70bbcfc6/src/kudu/security/tls_handshake-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/security/tls_handshake-test.cc b/src/kudu/security/tls_handshake-test.cc
index 5ee550a..337a7af 100644
--- a/src/kudu/security/tls_handshake-test.cc
+++ b/src/kudu/security/tls_handshake-test.cc
@@ -95,12 +95,10 @@ class TestTlsHandshake : public KuduTest {
 
 TEST_F(TestTlsHandshake, TestSuccessfulHandshake) {
   // Both client and server have certs and CA.
-  ASSERT_OK(client_tls_.LoadCertificate(cert_path_));
-  ASSERT_OK(client_tls_.LoadPrivateKey(key_path_));
   ASSERT_OK(client_tls_.LoadCertificateAuthority(cert_path_));
-  ASSERT_OK(server_tls_.LoadCertificate(cert_path_));
-  ASSERT_OK(server_tls_.LoadPrivateKey(key_path_));
+  ASSERT_OK(client_tls_.LoadCertificateAndKey(cert_path_, key_path_));
   ASSERT_OK(server_tls_.LoadCertificateAuthority(cert_path_));
+  ASSERT_OK(server_tls_.LoadCertificateAndKey(cert_path_, key_path_));
 
   TlsHandshake server;
   TlsHandshake client;
@@ -134,8 +132,7 @@ TEST_F(TestTlsHandshake, TestSuccessfulHandshake) {
 // Client has no cert.
 // Server has self-signed cert.
 TEST_F(TestTlsHandshake, Test_ClientNone_ServerSelfSigned) {
-  ASSERT_OK(server_tls_.LoadCertificate(cert_path_));
-  ASSERT_OK(server_tls_.LoadPrivateKey(key_path_));
+  ASSERT_OK(server_tls_.LoadCertificateAndKey(cert_path_, key_path_));
 
   // If the client wants to verify the server, it should fail because
   // the server cert is self-signed.
@@ -165,11 +162,9 @@ TEST_F(TestTlsHandshake, Test_ClientNone_ServerSelfSigned) {
 // which isn't very realistic. We should have this generate self-signed certs
 // on the fly.
 TEST_F(TestTlsHandshake, Test_ClientSelfSigned_ServerSelfSigned) {
-  ASSERT_OK(client_tls_.LoadCertificate(cert_path_));
-  ASSERT_OK(client_tls_.LoadPrivateKey(key_path_));
+  ASSERT_OK(client_tls_.LoadCertificateAndKey(cert_path_, key_path_));
   ASSERT_OK(client_tls_.LoadCertificateAuthority(cert_path_));
-  ASSERT_OK(server_tls_.LoadCertificate(cert_path_));
-  ASSERT_OK(server_tls_.LoadPrivateKey(key_path_));
+  ASSERT_OK(server_tls_.LoadCertificateAndKey(cert_path_, key_path_));
   ASSERT_OK(server_tls_.LoadCertificateAuthority(cert_path_));
 
   // This scenario should succeed in all cases.


[2/2] kudu git commit: security: simplify CertSigner interface

Posted by to...@apache.org.
security: simplify CertSigner interface

This makes CertSigner a "one-shot" type of instance, which no longer
holds long-lived references to CA certs or keys. This also removes the
'InitFromFiles()' path in CertSigner, since that's now doable directly
using the wrapper objects, and the code ended up only used by tests.

With these changes, a bunch of other things got simplified:
- no need for shared_ptr<PrivateKey> in a few places where it had leaked
- no need for multi-threading tests in cert_management-test

Change-Id: I50ee09a8bb6fba4ab6111288769b660ce61f048b
Reviewed-on: http://gerrit.cloudera.org:8080/5846
Tested-by: Kudu Jenkins
Reviewed-by: Dan Burkert <da...@apache.org>
Reviewed-by: Todd Lipcon <to...@apache.org>


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

Branch: refs/heads/master
Commit: 848180654d5954255b1ac4fac7a19b0a7ad669bd
Parents: 70bbcfc
Author: Todd Lipcon <to...@apache.org>
Authored: Tue Jan 31 16:55:35 2017 -0800
Committer: Todd Lipcon <to...@apache.org>
Committed: Fri Feb 3 17:32:33 2017 +0000

----------------------------------------------------------------------
 src/kudu/master/master_cert_authority.cc     |  23 +-
 src/kudu/master/master_cert_authority.h      |   5 +-
 src/kudu/security/ca/cert_management-test.cc | 252 ++++++----------------
 src/kudu/security/ca/cert_management.cc      |  96 ++-------
 src/kudu/security/ca/cert_management.h       |  54 ++---
 src/kudu/security/crypto.cc                  |   2 +-
 src/kudu/security/crypto.h                   |   2 +-
 src/kudu/security/security-test-util.cc      |  14 +-
 src/kudu/security/security-test-util.h       |   4 +-
 9 files changed, 123 insertions(+), 329 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/84818065/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 45190a1..fd24f28 100644
--- a/src/kudu/master/master_cert_authority.cc
+++ b/src/kudu/master/master_cert_authority.cc
@@ -27,9 +27,8 @@
 #include "kudu/security/openssl_util.h"
 #include "kudu/util/flag_tags.h"
 
-using std::make_shared;
-using std::shared_ptr;
 using std::string;
+using std::unique_ptr;
 
 using kudu::security::Cert;
 using kudu::security::CertSignRequest;
@@ -76,25 +75,19 @@ Status MasterCertAuthority::Init() {
   CHECK(!ca_private_key_);
 
   // Create a key and cert for the self-signed CA.
-  auto key = make_shared<PrivateKey>();
-  auto ca_cert = make_shared<Cert>();
-  RETURN_NOT_OK(GeneratePrivateKey(FLAGS_master_ca_rsa_key_length_bits,
-                                   key.get()));
-
-  RETURN_NOT_OK(CertSigner::SelfSignCA(key, PrepareCaConfig(server_uuid_), ca_cert.get()));
+  unique_ptr<PrivateKey> key(new PrivateKey());
+  unique_ptr<Cert> ca_cert(new Cert());
+  RETURN_NOT_OK(GeneratePrivateKey(FLAGS_master_ca_rsa_key_length_bits, key.get()));
+  RETURN_NOT_OK(CertSigner::SelfSignCA(*key, PrepareCaConfig(server_uuid_), ca_cert.get()));
 
   // Initialize our signer with the new CA.
-  auto signer = make_shared<CertSigner>();
-  RETURN_NOT_OK(signer->Init(ca_cert, key));
-
-  cert_signer_ = std::move(signer);
   ca_cert_ = std::move(ca_cert);
   ca_private_key_ = std::move(key);
   return Status::OK();
 }
 
 Status MasterCertAuthority::SignServerCSR(const string& csr_der, string* cert_der) {
-  CHECK(cert_signer_) << "not initialized";
+  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
@@ -104,8 +97,10 @@ Status MasterCertAuthority::SignServerCSR(const string& csr_der, string* cert_de
   CertSignRequest csr;
   RETURN_NOT_OK_PREPEND(csr.FromString(csr_der, security::DataFormat::DER),
                         "could not parse CSR");
+  // TODO(PKI): need to set expiration interval on the signed CA cert!
   Cert cert;
-  RETURN_NOT_OK_PREPEND(cert_signer_->Sign(csr, &cert),
+  RETURN_NOT_OK_PREPEND(CertSigner(ca_cert_.get(), ca_private_key_.get())
+                        .Sign(csr, &cert),
                         "failed to sign cert");
 
   RETURN_NOT_OK_PREPEND(cert.ToString(cert_der, security::DataFormat::DER),

http://git-wip-us.apache.org/repos/asf/kudu/blob/84818065/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 9b8e499..3ffed73 100644
--- a/src/kudu/master/master_cert_authority.h
+++ b/src/kudu/master/master_cert_authority.h
@@ -66,9 +66,8 @@ class MasterCertAuthority {
   // The UUID of the master. This is used as a field in the certificate.
   const std::string server_uuid_;
 
-  std::shared_ptr<security::ca::CertSigner> cert_signer_;
-  std::shared_ptr<security::PrivateKey> ca_private_key_;
-  std::shared_ptr<security::Cert> ca_cert_;
+  std::unique_ptr<security::PrivateKey> ca_private_key_;
+  std::unique_ptr<security::Cert> ca_cert_;
 
   DISALLOW_COPY_AND_ASSIGN(MasterCertAuthority);
 };

http://git-wip-us.apache.org/repos/asf/kudu/blob/84818065/src/kudu/security/ca/cert_management-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/security/ca/cert_management-test.cc b/src/kudu/security/ca/cert_management-test.cc
index 0a9b3a8..d296633 100644
--- a/src/kudu/security/ca/cert_management-test.cc
+++ b/src/kudu/security/ca/cert_management-test.cc
@@ -18,7 +18,6 @@
 #include "kudu/security/ca/cert_management.h"
 
 #include <memory>
-#include <thread>
 #include <utility>
 #include <vector>
 
@@ -29,16 +28,12 @@
 #include "kudu/security/openssl_util.h"
 #include "kudu/security/security-test-util.h"
 #include "kudu/security/test/test_certs.h"
-#include "kudu/util/env.h"
-#include "kudu/util/path_util.h"
 #include "kudu/util/status.h"
 #include "kudu/util/test_macros.h"
 #include "kudu/util/test_util.h"
 
-using std::shared_ptr;
 using std::string;
 using std::vector;
-using std::thread;
 using strings::Substitute;
 
 namespace kudu {
@@ -47,38 +42,15 @@ namespace ca {
 
 class CertManagementTest : public KuduTest {
  public:
-  CertManagementTest() :
-      pem_dir_(GetTestPath("pem")),
-      ca_cert_file_(JoinPathSegments(pem_dir_, "ca.cert.pem")),
-      ca_private_key_file_(JoinPathSegments(pem_dir_, "ca.pkey.pem")),
-      ca_public_key_file_(JoinPathSegments(pem_dir_, "ca.pubkey.pem")),
-      ca_exp_cert_file_(JoinPathSegments(pem_dir_, "ca.exp.cert.pem")),
-      ca_exp_private_key_file_(JoinPathSegments(pem_dir_, "ca.exp.pkey.pem")) {
-  }
-
   void SetUp() override {
-    ASSERT_OK(env_->CreateDir(pem_dir_));
-    ASSERT_OK(WriteStringToFile(env_, kCaCert, ca_cert_file_));
-    ASSERT_OK(WriteStringToFile(env_, kCaPrivateKey, ca_private_key_file_));
-    ASSERT_OK(WriteStringToFile(env_, kCaPublicKey, ca_public_key_file_));
-    ASSERT_OK(WriteStringToFile(env_, kCaExpiredCert, ca_exp_cert_file_));
-    ASSERT_OK(WriteStringToFile(env_, kCaExpiredPrivateKey,
-        ca_exp_private_key_file_));
+    ASSERT_OK(ca_cert_.FromString(kCaCert, DataFormat::PEM));
+    ASSERT_OK(ca_private_key_.FromString(kCaPrivateKey, DataFormat::PEM));
+    ASSERT_OK(ca_public_key_.FromString(kCaPublicKey, DataFormat::PEM));
+    ASSERT_OK(ca_exp_cert_.FromString(kCaExpiredCert, DataFormat::PEM));
+    ASSERT_OK(ca_exp_private_key_.FromString(kCaExpiredPrivateKey, DataFormat::PEM));
   }
 
  protected:
-  // Different sharing scenarios for request generator and signer.
-  enum SharingType {
-    DEDICATED,
-    SHARED
-  };
-
-  // Different init patterns for request generator and signer.
-  enum InitType {
-    SINGLE_INIT,
-    MULTIPLE_INIT
-  };
-
   CertRequestGenerator::Config PrepareConfig(
       const string& uuid,
       const vector<string>& hostnames = {},
@@ -100,76 +72,30 @@ class CertManagementTest : public KuduTest {
     };
   }
 
-  // Run multiple threads which do certificate signing request generation
-  // and signing those in parallel.  The 'is_shared' and 'multi_init' parameters
-  // are to specify whether the threads use shared
-  // CertRequestGenerator/CertSigner instances and whether every thread
-  // initializes the shared instance it's using.
-  void SignMultiThread(size_t num_threads, size_t iter_num,
-                       SharingType sharing_type, InitType init_type) {
-    const CertRequestGenerator::Config gen_config(
-        PrepareConfig("757F3158-DCB5-4D6C-8054-5348BB4AEA07",
-                      {"localhost"}, {"127.0.0.1"}));
-
-    CertRequestGenerator gen_shared(gen_config);
-    if (SINGLE_INIT == init_type) {
-      ASSERT_OK(gen_shared.Init());
-    }
-    CertSigner signer_shared;
-    if (SINGLE_INIT == init_type) {
-      ASSERT_OK(signer_shared.InitFromFiles(ca_cert_file_, ca_private_key_file_));
-    }
-
-    vector<thread> threads;
-    threads.reserve(num_threads);
-    for (size_t thread_idx = 0; thread_idx < num_threads; ++thread_idx) {
-      // 'thread_idx' is captured by value to avoid data races
-      threads.emplace_back([&, thread_idx]() {
-        for (size_t i = 0; i < iter_num; ++i) {
-          CertRequestGenerator gen_local(gen_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) {
-            CHECK_OK(gen.Init());
-          }
-          const size_t sel = i % 4;
-          const size_t key_bits = (sel + 1) * 512;
-          PrivateKey key;
-          CHECK_OK(GeneratePrivateKey(key_bits, &key));
-          CertSignRequest req;
-          CHECK_OK(gen.GenerateRequest(key, &req));
-          if (DEDICATED == sharing_type) {
-            CHECK_OK(signer.InitFromFiles(ca_cert_file_, ca_private_key_file_));
-          }
-          Cert cert;
-          CHECK_OK(signer.Sign(req, &cert));
-        }
-      });
-    }
-    for (auto& e : threads) {
-      e.join();
-    }
+  // Create a new private key in 'key' and return a CSR associated with that
+  // key.
+  template<class CSRGen = CertRequestGenerator>
+  CertSignRequest PrepareTestCSR(CertRequestGenerator::Config config,
+                                 PrivateKey* key) {
+    CHECK_OK(GeneratePrivateKey(512, key));
+    CSRGen gen(std::move(config));
+    CHECK_OK(gen.Init());
+    CertSignRequest req;
+    CHECK_OK(gen.GenerateRequest(*key, &req));
+    return req;
   }
 
-  const string pem_dir_;
-
-  const string ca_cert_file_;
-  const string ca_private_key_file_;
-  const string ca_public_key_file_;
+  Cert ca_cert_;
+  PrivateKey ca_private_key_;
+  PublicKey ca_public_key_;
 
-  const string ca_exp_cert_file_;
-  const string ca_exp_private_key_file_;
+  Cert ca_exp_cert_;
+  PrivateKey ca_exp_private_key_;
 };
 
 // Check input/output of RSA private keys in PEM format.
 TEST_F(CertManagementTest, RsaPrivateKeyInputOutputPEM) {
-  PrivateKey key;
-  ASSERT_OK(key.FromFile(ca_private_key_file_, DataFormat::PEM));
+  const auto& key = ca_private_key_;
   string key_str;
   key.ToString(&key_str, DataFormat::PEM);
   RemoveExtraWhitespace(&key_str);
@@ -181,8 +107,7 @@ TEST_F(CertManagementTest, RsaPrivateKeyInputOutputPEM) {
 
 // Check input/output of RSA public keys in PEM format.
 TEST_F(CertManagementTest, RsaPublicKeyInputOutputPEM) {
-  PublicKey key;
-  ASSERT_OK(key.FromFile(ca_public_key_file_, DataFormat::PEM));
+  const auto& key = ca_public_key_;
   string str_key;
   key.ToString(&str_key, DataFormat::PEM);
   RemoveExtraWhitespace(&str_key);
@@ -195,8 +120,7 @@ TEST_F(CertManagementTest, RsaPublicKeyInputOutputPEM) {
 // Check extraction of the public part out from RSA private keys par.
 TEST_F(CertManagementTest, RsaExtractPublicPartFromPrivateKey) {
   // Load the reference RSA private key.
-  PrivateKey private_key;
-  ASSERT_OK(private_key.FromString(kCaPrivateKey, DataFormat::PEM));
+  const PrivateKey& private_key = ca_private_key_;
 
   PublicKey public_key;
   ASSERT_OK(private_key.GetPublicKey(&public_key));
@@ -211,8 +135,7 @@ TEST_F(CertManagementTest, RsaExtractPublicPartFromPrivateKey) {
 
 // Check input/output of the X509 certificates in PEM format.
 TEST_F(CertManagementTest, CertInputOutputPEM) {
-  Cert cert;
-  ASSERT_OK(cert.FromFile(ca_cert_file_, DataFormat::PEM));
+  const Cert& cert = ca_cert_;
   string cert_str;
   cert.ToString(&cert_str, DataFormat::PEM);
   RemoveExtraWhitespace(&cert_str);
@@ -222,6 +145,15 @@ TEST_F(CertManagementTest, CertInputOutputPEM) {
   EXPECT_EQ(ca_input_cert, cert_str);
 }
 
+// Check that Cert behaves in a predictable way if given invalid PEM data.
+TEST_F(CertManagementTest, CertInvalidInput) {
+  // 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.
+  Cert c;
+  ASSERT_FALSE(c.FromFile("/bin/sh", DataFormat::PEM).ok());
+}
+
 // Check for basic SAN-related constraints while initializing
 // CertRequestGenerator objects.
 TEST_F(CertManagementTest, RequestGeneratorSanConstraints) {
@@ -280,7 +212,6 @@ TEST_F(CertManagementTest, RequestGeneratorBasics) {
 
   PrivateKey key;
   ASSERT_OK(GeneratePrivateKey(1024, &key));
-  ASSERT_OK(GeneratePrivateKey(2048, &key));
   CertRequestGenerator gen(gen_config);
   ASSERT_OK(gen.Init());
   string key_str;
@@ -290,29 +221,24 @@ TEST_F(CertManagementTest, RequestGeneratorBasics) {
   ASSERT_TRUE(s.IsRuntimeError());
 }
 
-// Check that CertSigner behaves in a predictable way if given non-expected
-// content for the CA private key/certificate.
-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.
-  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) {
+  PrivateKey key;
+  const auto& csr = PrepareTestCSR(PrepareConfig("test-uuid", {"localhost"}), &key);
   {
-    CertSigner signer;
-    Status s = signer.InitFromFiles(ca_cert_file_, ca_exp_private_key_file_);
+    Cert cert;
+    Status s = CertSigner(&ca_cert_, &ca_exp_private_key_)
+        .Sign(csr, &cert);
+
     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");
   }
   {
-    CertSigner signer;
-    Status s = signer.InitFromFiles(ca_exp_cert_file_, ca_private_key_file_);
+    Cert cert;
+    Status s = CertSigner(&ca_exp_cert_, &ca_private_key_)
+        .Sign(csr, &cert);
     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");
@@ -325,34 +251,21 @@ TEST_F(CertManagementTest, SignerInitWithExpiredCert) {
   const CertRequestGenerator::Config gen_config(
       PrepareConfig("F4466090-BBF8-4042-B72F-BB257500C45A", {"localhost"}));
   PrivateKey key;
-  ASSERT_OK(GeneratePrivateKey(2048, &key));
-  CertRequestGenerator gen(gen_config);
-  ASSERT_OK(gen.Init());
-  CertSignRequest req;
-  ASSERT_OK(gen.GenerateRequest(key, &req));
-  CertSigner signer;
-  // Even if the certificate is expired, the signer should initialize OK.
-  ASSERT_OK(signer.InitFromFiles(ca_exp_cert_file_, ca_exp_private_key_file_));
-  Cert cert;
+  CertSignRequest req = PrepareTestCSR(gen_config, &key);
+
   // Signer works fine even with expired CA certificate.
-  ASSERT_OK(signer.Sign(req, &cert));
+  Cert cert;
+  ASSERT_OK(CertSigner(&ca_exp_cert_, &ca_exp_private_key_).Sign(req, &cert));
 }
 
-// Generate X509 CSR and issues corresponding certificate: everything is done
-// in a single-threaded fashion.
+// Generate X509 CSR and issues corresponding certificate.
 TEST_F(CertManagementTest, SignCert) {
   const CertRequestGenerator::Config gen_config(
       PrepareConfig("test-uuid", {"localhost"}, {"127.0.0.1", "127.0.10.20"}));
   PrivateKey key;
-  ASSERT_OK(GeneratePrivateKey(2048, &key));
-  CertRequestGenerator gen(gen_config);
-  ASSERT_OK(gen.Init());
-  CertSignRequest req;
-  ASSERT_OK(gen.GenerateRequest(key, &req));
-  CertSigner signer;
-  ASSERT_OK(signer.InitFromFiles(ca_cert_file_, ca_private_key_file_));
+  const auto& csr = PrepareTestCSR(gen_config, &key);
   Cert cert;
-  ASSERT_OK(signer.Sign(req, &cert));
+  ASSERT_OK(CertSigner(&ca_cert_, &ca_private_key_).Sign(csr, &cert));
 
   EXPECT_EQ("C = US, ST = CA, O = MyCompany, CN = MyName, emailAddress = my@email.com",
             cert.IssuerName());
@@ -365,45 +278,28 @@ TEST_F(CertManagementTest, SignCaCert) {
   const CertRequestGenerator::Config gen_config(
       PrepareConfig("8C084CF6-A30B-4F5B-9673-A73E62E29A9D"));
   PrivateKey key;
-  ASSERT_OK(GeneratePrivateKey(2048, &key));
-  CaCertRequestGenerator gen(gen_config);
-  ASSERT_OK(gen.Init());
-  CertSignRequest req;
-  ASSERT_OK(gen.GenerateRequest(key, &req));
-  CertSigner signer;
-  ASSERT_OK(signer.InitFromFiles(ca_cert_file_, ca_private_key_file_));
+  const auto& csr = PrepareTestCSR<CaCertRequestGenerator>(gen_config, &key);
   Cert cert;
-  ASSERT_OK(signer.Sign(req, &cert));
+  ASSERT_OK(CertSigner(&ca_cert_, &ca_private_key_).Sign(csr, &cert));
 }
 
 // Test the creation and use of a CA which uses a self-signed CA cert
 // generated on the fly.
 TEST_F(CertManagementTest, TestSelfSignedCA) {
-  shared_ptr<PrivateKey> ca_key;
-  shared_ptr<Cert> ca_cert;
+  PrivateKey ca_key;
+  Cert ca_cert;
   ASSERT_OK(GenerateSelfSignedCAForTests(&ca_key, &ca_cert));
 
-  // Create a key for the tablet server.
-  auto ts_key = std::make_shared<PrivateKey>();
-  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));
-  }
+  // Create a key and CSR for the tablet server.
+  const auto& config = PrepareConfig(
+      "some-tablet-server",
+      {"localhost"}, {"127.0.0.1", "127.0.10.20"});
+  PrivateKey ts_key;
+  CertSignRequest ts_csr = PrepareTestCSR(config, &ts_key);
 
   // 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));
-  }
+  ASSERT_OK(CertSigner(&ca_cert, &ca_key).Sign(ts_csr, &ts_cert));
 }
 
 // Check the transformation chains for X509 CSRs:
@@ -446,10 +342,9 @@ TEST_F(CertManagementTest, X509FromAndToString) {
   CertSignRequest req;
   ASSERT_OK(gen.GenerateRequest(key, &req));
 
-  CertSigner signer;
-  ASSERT_OK(signer.InitFromFiles(ca_cert_file_, ca_private_key_file_));
   Cert cert_ref;
-  ASSERT_OK(signer.Sign(req, &cert_ref));
+  ASSERT_OK(CertSigner(&ca_cert_, &ca_private_key_)
+            .Sign(req, &cert_ref));
 
   for (auto format : kFormats) {
     SCOPED_TRACE(Substitute("X509 format: $0", DataFormatToString(format)));
@@ -463,27 +358,6 @@ 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 earlier (i.e. those threads do not call Init()).
-TEST_F(CertManagementTest, SignMultiThreadExclusiveSingleInit) {
-  ASSERT_NO_FATAL_FAILURE(SignMultiThread(32, 16, DEDICATED, SINGLE_INIT));
-}
-
-// Generate CSR and issue corresponding certificate in a multi-threaded fashion:
-// every thread uses its own instances of CertRequestGenerator and CertSigner,
-// and every thread initializes those shared objects by itself.
-TEST_F(CertManagementTest, SignMultiThreadExclusiveMultiInit) {
-  ASSERT_NO_FATAL_FAILURE(SignMultiThread(16, 32, DEDICATED, MULTIPLE_INIT));
-}
-
-// Generate CSR and issue corresponding certificate in a multi-thread fashion:
-// all threads use shared instances of CertRequestGenerator and CertSigner,
-// 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));
-}
-
 } // namespace ca
 } // namespace security
 } // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/84818065/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 710b23e..a871b75 100644
--- a/src/kudu/security/ca/cert_management.cc
+++ b/src/kudu/security/ca/cert_management.cc
@@ -44,7 +44,6 @@
 using std::lock_guard;
 using std::move;
 using std::ostringstream;
-using std::shared_ptr;
 using std::string;
 using strings::Substitute;
 
@@ -281,7 +280,7 @@ Status CaCertRequestGenerator::SetExtensions(X509_REQ* req) const {
   return Status::OK();
 }
 
-Status CertSigner::SelfSignCA(const shared_ptr<PrivateKey>& key,
+Status CertSigner::SelfSignCA(const PrivateKey& key,
                               CaCertRequestGenerator::Config config,
                               Cert* cert) {
   // Generate a CSR for the CA.
@@ -289,90 +288,37 @@ Status CertSigner::SelfSignCA(const shared_ptr<PrivateKey>& key,
   {
     CaCertRequestGenerator gen(std::move(config));
     RETURN_NOT_OK(gen.Init());
-    RETURN_NOT_OK(gen.GenerateRequest(*key, &ca_csr));
+    RETURN_NOT_OK(gen.GenerateRequest(key, &ca_csr));
   }
 
   // Self-sign the CA's CSR.
-  {
-    CertSigner ca_signer;
-    RETURN_NOT_OK(ca_signer.InitForSelfSigning(key));
-    RETURN_NOT_OK(ca_signer.Sign(ca_csr, cert));
-  }
+  RETURN_NOT_OK(CertSigner(nullptr, &key).Sign(ca_csr, cert));
   return Status::OK();
 }
 
-CertSigner::CertSigner()
-    : is_initialized_(false) {
-}
-
-Status CertSigner::Init(shared_ptr<Cert> ca_cert,
-                        shared_ptr<PrivateKey> ca_private_key) {
-  CHECK(ca_cert && ca_cert->GetRawData());
+CertSigner::CertSigner(const Cert* ca_cert,
+                       const PrivateKey* ca_private_key)
+    : ca_cert_(ca_cert),
+      ca_private_key_(ca_private_key) {
+  // Private key is required.
   CHECK(ca_private_key && ca_private_key->GetRawData());
-  InitializeOpenSSL();
-
-  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<PrivateKey> 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>();
-  std::shared_ptr<PrivateKey> key = std::make_shared<PrivateKey>();
-  RETURN_NOT_OK(cert->FromFile(ca_cert_path, DataFormat::PEM));
-  RETURN_NOT_OK(key->FromFile(ca_private_key_path,
-                              DataFormat::PEM));
-  OPENSSL_RET_NOT_OK(
-      X509_check_private_key(cert->GetRawData(), key->GetRawData()),
-      Substitute("$0, $1: CA certificate and private key do not match",
-          ca_cert_path, ca_private_key_path));
-  ca_cert_ = std::move(cert);
-  ca_private_key_ = std::move(key);
-  is_initialized_ = true;
-  return Status::OK();
-}
-
-bool CertSigner::Initialized() const {
-  lock_guard<simple_spinlock> guard(lock_);
-  return is_initialized_;
-}
-
-const Cert& CertSigner::ca_cert() const {
-  lock_guard<simple_spinlock> guard(lock_);
-  DCHECK(is_initialized_);
-  return *CHECK_NOTNULL(ca_cert_.get());
-}
-
-const PrivateKey& CertSigner::ca_private_key() const {
-  lock_guard<simple_spinlock> guard(lock_);
-  DCHECK(is_initialized_);
-  return *ca_private_key_;
+  // The cert is optional, but if we have it, it should be initialized.
+  CHECK(!ca_cert || ca_cert->GetRawData());
 }
 
 Status CertSigner::Sign(const CertSignRequest& req, Cert* ret) const {
+  InitializeOpenSSL();
   CHECK(ret);
-  CHECK(Initialized());
+
+  // If we are not self-signing, then make sure that the provided CA
+  // cert and key match each other. Technically this would be programmer
+  // error since we're always using internally-generated CA certs, but
+  // this isn't a hot path so we'll keep the extra safety.
+  if (ca_cert_) {
+    OPENSSL_RET_NOT_OK(
+        X509_check_private_key(ca_cert_->GetRawData(), ca_private_key_->GetRawData()),
+        "CA certificate and private key do not match");
+  }
   auto x509 = ssl_make_unique(X509_new());
   RETURN_NOT_OK(FillCertTemplateFromRequest(req.GetRawData(), x509.get()));
   RETURN_NOT_OK(DoSign(EVP_sha256(), exp_interval_sec_, x509.get()));

http://git-wip-us.apache.org/repos/asf/kudu/blob/84818065/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 7a2887a..203a810 100644
--- a/src/kudu/security/ca/cert_management.h
+++ b/src/kudu/security/ca/cert_management.h
@@ -132,69 +132,59 @@ class CaCertRequestGenerator : public CertRequestGeneratorBase {
 };
 
 // An utility class for issuing and signing certificates.
+//
+// This is used in "fluent" style. For example:
+//
+//    CHECK_OK(CertSigner(&my_ca_cert, &my_ca_key)
+//      .set_expiration_interval(MonoDelta::FromSeconds(3600))
+//      .Sign(csr, &cert));
+//
+// As such, this class is not guaranteed thread-safe.
 class CertSigner {
  public:
   // Generate a self-signed certificate authority using the given key
   // and CSR configuration.
-  static Status SelfSignCA(const std::shared_ptr<PrivateKey>& key,
+  static Status SelfSignCA(const PrivateKey& key,
                            CaCertRequestGenerator::Config config,
                            Cert* cert);
+
   // Create a CertSigner.
-  // Exactly one of the following Init*() methods must be called
-  // exactly once before the instance may be used.
-  CertSigner();
+  //
+  // The given cert and key must stay valid for the lifetime of the
+  // cert signer. See class documentation above for recommended usage.
+  //
+  // 'ca_cert' may be nullptr in order to perform self-signing (though
+  // the SelfSignCA() static method above is recommended).
+  CertSigner(const Cert* ca_cert, const PrivateKey* ca_private_key);
   ~CertSigner() = default;
 
-  // 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<PrivateKey> 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);
-
   // 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_);
+  CertSigner& set_expiration_interval(MonoDelta expiration) {
     exp_interval_sec_ = expiration.ToSeconds();
+    return *this;
   }
 
-  bool Initialized() const;
-
-  const Cert& ca_cert() const;
-  const PrivateKey& ca_private_key() const;
-
   Status Sign(const CertSignRequest& req, Cert* ret) const;
 
  private:
+
   static Status CopyExtensions(X509_REQ* req, X509* x);
   static Status FillCertTemplateFromRequest(X509_REQ* req, X509* tmpl);
   static Status DigestSign(const EVP_MD* md, EVP_PKEY* pkey, X509* x);
   static Status GenerateSerial(c_unique_ptr<ASN1_INTEGER>* ret);
 
-  // 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<PrivateKey> private_key);
-
   Status DoSign(const EVP_MD* digest, int32_t exp_seconds, X509 *ret) const;
 
-  mutable simple_spinlock lock_;
-  bool is_initialized_; // protected by lock_
-
   // 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_;
+  const Cert* const ca_cert_;
 
   // The CA private key. If configured for self-signing, this is the
   // private key associated with the target cert.
-  std::shared_ptr<PrivateKey> ca_private_key_;
+  const PrivateKey* const ca_private_key_;
 
   DISALLOW_COPY_AND_ASSIGN(CertSigner);
 };

http://git-wip-us.apache.org/repos/asf/kudu/blob/84818065/src/kudu/security/crypto.cc
----------------------------------------------------------------------
diff --git a/src/kudu/security/crypto.cc b/src/kudu/security/crypto.cc
index c5d5c8c..dda9da3 100644
--- a/src/kudu/security/crypto.cc
+++ b/src/kudu/security/crypto.cc
@@ -169,7 +169,7 @@ Status PrivateKey::FromFile(const std::string& fpath, DataFormat format) {
 // The code is modeled after $OPENSSL_ROOT/apps/rsa.c code: there is
 // corresponding functionality to read public part from RSA private/public
 // keypair.
-Status PrivateKey::GetPublicKey(PublicKey* public_key) {
+Status PrivateKey::GetPublicKey(PublicKey* public_key) const {
   CHECK(public_key);
   auto rsa = ssl_make_unique(EVP_PKEY_get1_RSA(CHECK_NOTNULL(data_.get())));
   if (PREDICT_FALSE(!rsa)) {

http://git-wip-us.apache.org/repos/asf/kudu/blob/84818065/src/kudu/security/crypto.h
----------------------------------------------------------------------
diff --git a/src/kudu/security/crypto.h b/src/kudu/security/crypto.h
index 6063f37..ce951b9 100644
--- a/src/kudu/security/crypto.h
+++ b/src/kudu/security/crypto.h
@@ -69,7 +69,7 @@ class PrivateKey : public RawDataWrapper<EVP_PKEY> {
   Status FromFile(const std::string& fpath, DataFormat format);
 
   // Output the public part of the keypair into the specified placeholder.
-  Status GetPublicKey(PublicKey* public_key);
+  Status GetPublicKey(PublicKey* public_key) const;
 
   // Using the key, generate data signature using the specified
   // message digest algorithm. The result signature is in raw format

http://git-wip-us.apache.org/repos/asf/kudu/blob/84818065/src/kudu/security/security-test-util.cc
----------------------------------------------------------------------
diff --git a/src/kudu/security/security-test-util.cc b/src/kudu/security/security-test-util.cc
index 2fa8b1f..c01f562 100644
--- a/src/kudu/security/security-test-util.cc
+++ b/src/kudu/security/security-test-util.cc
@@ -17,8 +17,6 @@
 
 #include "kudu/security/security-test-util.h"
 
-#include <memory>
-
 #include <glog/logging.h>
 
 #include "kudu/security/ca/cert_management.h"
@@ -31,19 +29,13 @@ namespace security {
 using ca::CaCertRequestGenerator;
 using ca::CertSigner;
 
-Status GenerateSelfSignedCAForTests(std::shared_ptr<PrivateKey>* ca_key,
-                                    std::shared_ptr<Cert>* ca_cert) {
+Status GenerateSelfSignedCAForTests(PrivateKey* ca_key, Cert* ca_cert) {
   // Create a key for the self-signed CA.
-  auto ret_ca_key = std::make_shared<PrivateKey>();
-  auto ret_ca_cert = std::make_shared<Cert>();
-  RETURN_NOT_OK(GeneratePrivateKey(512, ret_ca_key.get()));
+  RETURN_NOT_OK(GeneratePrivateKey(512, ca_key));
 
   CaCertRequestGenerator::Config config;
   config.uuid = "test-ca-uuid";
-  RETURN_NOT_OK(CertSigner::SelfSignCA(ret_ca_key, config, ret_ca_cert.get()));
-
-  *ca_key = std::move(ret_ca_key);
-  *ca_cert = std::move(ret_ca_cert);
+  RETURN_NOT_OK(CertSigner::SelfSignCA(*ca_key, config, ca_cert));
   return Status::OK();
 }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/84818065/src/kudu/security/security-test-util.h
----------------------------------------------------------------------
diff --git a/src/kudu/security/security-test-util.h b/src/kudu/security/security-test-util.h
index 17c5375..df4730f 100644
--- a/src/kudu/security/security-test-util.h
+++ b/src/kudu/security/security-test-util.h
@@ -99,9 +99,7 @@ dc+JVPKL8Fe4a8fmsI6ndcZQ9qpOdZM5WOD0ldKRc+SsrYKkTmOOJQ==
   return Status::OK();
 }
 
-// TODO(todd): change these from shared_ptrs to unique_ptrs
-Status GenerateSelfSignedCAForTests(std::shared_ptr<PrivateKey>* ca_key,
-                                    std::shared_ptr<Cert>* ca_cert);
+Status GenerateSelfSignedCAForTests(PrivateKey* ca_key, Cert* ca_cert);
 
 } // namespace security
 } // namespace kudu