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/01 01:12:51 UTC

[1/3] kudu git commit: Upgrade Snappy to 1.1.4

Repository: kudu
Updated Branches:
  refs/heads/master 99b9bf3b4 -> d91313d61


Upgrade Snappy to 1.1.4

In my random perusal of github, I noticed that we are a few point
releases out of date on Snappy. The release notes indicate that
compression should be ~5% faster and decompression 20% faster in 1.1.4.
v1.1.1 also claims to have increased decompression by 13-20% in some
particular benchmark cases.

Seems worth upgrading given the one-character diff.

Change-Id: Ieab52cc0c968f17b6a44328d49a42872284968bb
Reviewed-on: http://gerrit.cloudera.org:8080/5833
Tested-by: Todd Lipcon <to...@apache.org>
Reviewed-by: Mike Percy <mp...@apache.org>


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

Branch: refs/heads/master
Commit: e2cf460307d70f8cfd40a1a42e6c1133177b27ef
Parents: 99b9bf3
Author: Todd Lipcon <to...@apache.org>
Authored: Tue Jan 31 10:24:11 2017 -0800
Committer: Todd Lipcon <to...@apache.org>
Committed: Tue Jan 31 23:55:53 2017 +0000

----------------------------------------------------------------------
 thirdparty/vars.sh | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/e2cf4603/thirdparty/vars.sh
----------------------------------------------------------------------
diff --git a/thirdparty/vars.sh b/thirdparty/vars.sh
index 9d7f4ba..6f0178d 100644
--- a/thirdparty/vars.sh
+++ b/thirdparty/vars.sh
@@ -58,7 +58,7 @@ CMAKE_VERSION=3.6.1
 CMAKE_NAME=cmake-$CMAKE_VERSION
 CMAKE_SOURCE=$TP_SOURCE_DIR/$CMAKE_NAME
 
-SNAPPY_VERSION=1.1.0
+SNAPPY_VERSION=1.1.4
 SNAPPY_NAME=snappy-$SNAPPY_VERSION
 SNAPPY_SOURCE=$TP_SOURCE_DIR/$SNAPPY_NAME
 


[3/3] kudu git commit: [security] method to extract public part of an RSA key

Posted by to...@apache.org.
[security] method to extract public part of an RSA key

Added a method to get the public part of an RSA keypair.
Split the Key class into PublicKey and PrivateKey.  Reorganized the
layout of files in the src/security sub-directory: separated the common
crypto stuff from openssl_util.h into crypto.h.

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


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

Branch: refs/heads/master
Commit: d91313d6147bf3214d8954d810a2d4f6f4b5e123
Parents: e2cf460
Author: Alexey Serbin <as...@cloudera.com>
Authored: Tue Jan 24 15:15:13 2017 -0800
Committer: Todd Lipcon <to...@apache.org>
Committed: Wed Feb 1 01:08:26 2017 +0000

----------------------------------------------------------------------
 src/kudu/master/master_cert_authority.cc       |   7 +-
 src/kudu/master/master_cert_authority.h        |   6 +-
 src/kudu/security/CMakeLists.txt               |   4 +-
 src/kudu/security/ca/cert_management-test.cc   | 562 ++++++++++++++++++++
 src/kudu/security/ca/cert_management.cc        |  84 +--
 src/kudu/security/ca/cert_management.h         |  17 +-
 src/kudu/security/cert.cc                      |  74 +++
 src/kudu/security/cert.h                       |  49 ++
 src/kudu/security/crypto.cc                    | 164 ++++++
 src/kudu/security/crypto.h                     |  66 +++
 src/kudu/security/openssl_util.cc              | 186 -------
 src/kudu/security/openssl_util.h               |  99 +---
 src/kudu/security/openssl_util_bio.h           | 113 ++++
 src/kudu/security/server_cert_manager.cc       |   8 +-
 src/kudu/security/server_cert_manager.h        |   4 +-
 src/kudu/security/test/cert_management-test.cc | 493 -----------------
 src/kudu/security/test/test_certs.cc           |  26 +
 src/kudu/security/test/test_certs.h            |   4 +
 src/kudu/security/tls_handshake-test.cc        |   2 +
 src/kudu/security/tls_handshake.h              |   1 +
 src/kudu/security/tls_socket.cc                |   1 +
 src/kudu/security/token_signer.cc              |  16 +-
 src/kudu/security/token_signing_key.cc         |   6 +-
 src/kudu/security/token_signing_key.h          |   9 +-
 24 files changed, 1177 insertions(+), 824 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/d91313d6/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 54816eb..8b33991 100644
--- a/src/kudu/master/master_cert_authority.cc
+++ b/src/kudu/master/master_cert_authority.cc
@@ -22,15 +22,18 @@
 #include <string>
 
 #include "kudu/security/ca/cert_management.h"
+#include "kudu/security/cert.h"
+#include "kudu/security/crypto.h"
 #include "kudu/security/openssl_util.h"
 #include "kudu/util/flag_tags.h"
 
 using std::make_shared;
+using std::shared_ptr;
 using std::string;
 
 using kudu::security::Cert;
 using kudu::security::CertSignRequest;
-using kudu::security::Key;
+using kudu::security::PrivateKey;
 using kudu::security::ca::CaCertRequestGenerator;
 using kudu::security::ca::CertSigner;
 
@@ -73,7 +76,7 @@ Status MasterCertAuthority::Init() {
   CHECK(!ca_private_key_);
 
   // Create a key for the self-signed CA.
-  auto key = make_shared<Key>();
+  shared_ptr<PrivateKey> key(make_shared<PrivateKey>());
   RETURN_NOT_OK(GeneratePrivateKey(FLAGS_master_ca_rsa_key_length_bits,
                                    key.get()));
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/d91313d6/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 c3430c5..9b8e499 100644
--- a/src/kudu/master/master_cert_authority.h
+++ b/src/kudu/master/master_cert_authority.h
@@ -25,8 +25,10 @@
 namespace kudu {
 
 namespace security {
+
 class Cert;
-class Key;
+class PrivateKey;
+
 namespace ca {
 class CertSigner;
 } // namespace ca
@@ -65,7 +67,7 @@ class MasterCertAuthority {
   const std::string server_uuid_;
 
   std::shared_ptr<security::ca::CertSigner> cert_signer_;
-  std::shared_ptr<security::Key> ca_private_key_;
+  std::shared_ptr<security::PrivateKey> ca_private_key_;
   std::shared_ptr<security::Cert> ca_cert_;
 
   DISALLOW_COPY_AND_ASSIGN(MasterCertAuthority);

http://git-wip-us.apache.org/repos/asf/kudu/blob/d91313d6/src/kudu/security/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/kudu/security/CMakeLists.txt b/src/kudu/security/CMakeLists.txt
index b3ec6a7..6b48d27 100644
--- a/src/kudu/security/CMakeLists.txt
+++ b/src/kudu/security/CMakeLists.txt
@@ -56,6 +56,8 @@ endif()
 
 set(SECURITY_SRCS
   ca/cert_management.cc
+  cert.cc
+  crypto.cc
   init.cc
   openssl_util.cc
   ${PORTED_X509_CHECK_HOST_CC}
@@ -103,7 +105,7 @@ if (NOT NO_TESTS)
     security-test
     ${KUDU_MIN_TEST_LIBS})
 
-  ADD_KUDU_TEST(test/cert_management-test)
+  ADD_KUDU_TEST(ca/cert_management-test)
   ADD_KUDU_TEST(test/mini_kdc-test)
   ADD_KUDU_TEST(tls_handshake-test)
   ADD_KUDU_TEST(token-test)

http://git-wip-us.apache.org/repos/asf/kudu/blob/d91313d6/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
new file mode 100644
index 0000000..ff070f6
--- /dev/null
+++ b/src/kudu/security/ca/cert_management-test.cc
@@ -0,0 +1,562 @@
+// 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/security/ca/cert_management.h"
+
+#include <thread>
+#include <utility>
+#include <vector>
+
+#include "kudu/gutil/strings/strip.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/gutil/strings/util.h"
+#include "kudu/security/cert.h"
+#include "kudu/security/openssl_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::string;
+using std::vector;
+using std::thread;
+using strings::Substitute;
+
+namespace kudu {
+namespace security {
+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")),
+      ca_exp_public_key_file_(JoinPathSegments(pem_dir_, "ca.exp.pubkey.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(WriteStringToFile(env_, kCaExpiredPublicKey,
+        ca_exp_public_key_file_));
+  }
+
+ 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 = {},
+      const vector<string>& ips = {}) const {
+    const ::testing::TestInfo* const test_info =
+        ::testing::UnitTest::GetInstance()->current_test_info();
+    const string comment = string(test_info->test_case_name()) + "." +
+      test_info->name();
+    const CertRequestGenerator::Config config = {
+      "US",               // country
+      "CA",               // state
+      "San Francisco",    // locality
+      "ASF",              // org
+      "The Kudu Project", // unit
+      uuid,               // uuid
+      comment,            // comment
+      hostnames,          // hostnames
+      ips,                // ips
+    };
+    return config;
+  }
+
+  // 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();
+    }
+  }
+
+  const string pem_dir_;
+
+  const string ca_cert_file_;
+  const string ca_private_key_file_;
+  const string ca_public_key_file_;
+
+  const string ca_exp_cert_file_;
+  const string ca_exp_private_key_file_;
+  const string ca_exp_public_key_file_;
+};
+
+// 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));
+  string key_str;
+  key.ToString(&key_str, DataFormat::PEM);
+  RemoveExtraWhitespace(&key_str);
+
+  string ca_input_key(kCaPrivateKey);
+  RemoveExtraWhitespace(&ca_input_key);
+  EXPECT_EQ(ca_input_key, key_str);
+}
+
+// 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));
+  string str_key;
+  key.ToString(&str_key, DataFormat::PEM);
+  RemoveExtraWhitespace(&str_key);
+
+  string ref_str_key(kCaPublicKey);
+  RemoveExtraWhitespace(&ref_str_key);
+  EXPECT_EQ(ref_str_key, str_key);
+}
+
+// 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));
+
+  PublicKey public_key;
+  ASSERT_OK(private_key.GetPublicKey(&public_key));
+  string str_public_key;
+  public_key.ToString(&str_public_key, DataFormat::PEM);
+  RemoveExtraWhitespace(&str_public_key);
+
+  string ref_str_public_key(kCaPublicKey);
+  RemoveExtraWhitespace(&ref_str_public_key);
+  EXPECT_EQ(ref_str_public_key, str_public_key);
+}
+
+// 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));
+  string cert_str;
+  cert.ToString(&cert_str, DataFormat::PEM);
+  RemoveExtraWhitespace(&cert_str);
+
+  string ca_input_cert(kCaCert);
+  RemoveExtraWhitespace(&ca_input_cert);
+  EXPECT_EQ(ca_input_cert, cert_str);
+}
+
+// Check for basic SAN-related constraints while initializing
+// CertRequestGenerator objects.
+TEST_F(CertManagementTest, RequestGeneratorSanConstraints) {
+  const string kEntityUUID = "D94FBF10-6F40-4F9F-BC82-F96A1C4F2CFB";
+
+  // No hostnames, nor IP addresses are given to populate X509v3 SAN extension.
+  {
+    const CertRequestGenerator::Config gen_config = PrepareConfig(kEntityUUID);
+    CertRequestGenerator gen(gen_config);
+    const Status s = gen.Init();
+    const string err_msg = s.ToString();
+    ASSERT_TRUE(s.IsInvalidArgument()) << err_msg;
+    ASSERT_STR_CONTAINS(err_msg, "SAN: missing DNS names and IP addresses");
+  }
+
+  // An empty hostname
+  {
+    const CertRequestGenerator::Config gen_config =
+        PrepareConfig(kEntityUUID, {"localhost", ""});
+    CertRequestGenerator gen(gen_config);
+    const Status s = gen.Init();
+    const string err_msg = s.ToString();
+    ASSERT_TRUE(s.IsInvalidArgument()) << err_msg;
+    ASSERT_STR_CONTAINS(err_msg, "SAN: an empty hostname");
+  }
+
+  // An empty IP address
+  {
+    const CertRequestGenerator::Config gen_config =
+        PrepareConfig(kEntityUUID, {}, {"127.0.0.1", ""});
+    CertRequestGenerator gen(gen_config);
+    const Status s = gen.Init();
+    const string err_msg = s.ToString();
+    ASSERT_TRUE(s.IsInvalidArgument()) << err_msg;
+    ASSERT_STR_CONTAINS(err_msg, "SAN: an empty IP address");
+  }
+
+  // Missing UUID
+  {
+    const CertRequestGenerator::Config gen_config =
+        PrepareConfig("", {"localhost"});
+    CertRequestGenerator gen(gen_config);
+    const Status s = gen.Init();
+    const string err_msg = s.ToString();
+    ASSERT_TRUE(s.IsInvalidArgument()) << err_msg;
+    ASSERT_STR_CONTAINS(err_msg, "missing end-entity UUID/name");
+  }
+}
+
+// Check for the basic functionality of the CertRequestGenerator class:
+// check it's able to generate keys of expected number of bits and that it
+// reports an error if trying to generate a key of unsupported number of bits.
+TEST_F(CertManagementTest, RequestGeneratorBasics) {
+  const CertRequestGenerator::Config gen_config =
+      PrepareConfig("702C1C5E-CF02-4EDC-8883-07ECDEC8CE97", {"localhost"});
+
+  PrivateKey key;
+  ASSERT_OK(GeneratePrivateKey(1024, &key));
+  ASSERT_OK(GeneratePrivateKey(2048, &key));
+  CertRequestGenerator gen(gen_config);
+  ASSERT_OK(gen.Init());
+  string key_str;
+  key.ToString(&key_str, DataFormat::PEM);
+  // Check for non-supported number of bits for the key.
+  Status s = GeneratePrivateKey(7, &key);
+  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) {
+  {
+    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");
+  }
+  {
+    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");
+  }
+}
+
+// Check how CertSigner behaves if given expired CA certificate
+// and corresponding private key.
+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;
+  // Signer works fine even with expired CA certificate.
+  ASSERT_OK(signer.Sign(req, &cert));
+}
+
+// Generate X509 CSR and issues corresponding certificate: everything is done
+// 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"}));
+  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_));
+  Cert cert;
+  ASSERT_OK(signer.Sign(req, &cert));
+}
+
+// Generate X509 CA CSR and sign the result certificate.
+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_));
+  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<PrivateKey>();
+  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<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));
+  }
+
+  // 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 RSA private keys:
+//   internal -> PEM -> internal -> PEM
+//   internal -> DER -> internal -> DER
+TEST_F(CertManagementTest, RsaPrivateKeyFromAndToString) {
+  static const DataFormat kFormats[] = { DataFormat::PEM, DataFormat::DER };
+  static const uint16_t kKeyBits[] = { 256, 512, 1024, 2048, 3072, 4096 };
+
+  for (auto format : kFormats) {
+    for (auto key_bits : kKeyBits) {
+      SCOPED_TRACE(Substitute("key format: $0, key bits: $1",
+                              format == DataFormat::PEM ? "PEM"
+                                                        : "DER", key_bits));
+      PrivateKey key_ref;
+      ASSERT_OK(GeneratePrivateKey(key_bits, &key_ref));
+      string str_key_ref;
+      ASSERT_OK(key_ref.ToString(&str_key_ref, format));
+      PrivateKey key;
+      ASSERT_OK(key.FromString(str_key_ref, format));
+      string str_key;
+      ASSERT_OK(key.ToString(&str_key, format));
+      ASSERT_EQ(str_key_ref, str_key);
+    }
+  }
+}
+
+// Check the transformation chains for RSA public keys:
+//   internal -> PEM -> internal -> PEM
+//   internal -> DER -> internal -> DER
+TEST_F(CertManagementTest, RsaPublicKeyFromAndToString) {
+  static const DataFormat kFormats[] = { DataFormat::PEM, DataFormat::DER };
+  static const uint16_t kKeyBits[] = { 256, 512, 1024, 2048, 3072, 4096 };
+
+  for (auto format : kFormats) {
+    for (auto key_bits : kKeyBits) {
+      SCOPED_TRACE(Substitute("key format: $0, key bits: $1",
+                              format == DataFormat::PEM ? "PEM"
+                                                        : "DER", key_bits));
+      // Generate private RSA key.
+      PrivateKey private_key;
+      ASSERT_OK(GeneratePrivateKey(key_bits, &private_key));
+
+      // Extract public part of the key
+      PublicKey key_ref;
+      ASSERT_OK(private_key.GetPublicKey(&key_ref));
+
+      string str_key_ref;
+      ASSERT_OK(key_ref.ToString(&str_key_ref, format));
+      PublicKey key;
+      ASSERT_OK(key.FromString(str_key_ref, format));
+      string str_key;
+      ASSERT_OK(key.ToString(&str_key, format));
+      ASSERT_EQ(str_key_ref, str_key);
+    }
+  }
+}
+
+// Check the transformation chains for X509 CSRs:
+//   internal -> PEM -> internal -> PEM
+//   internal -> DER -> internal -> DER
+TEST_F(CertManagementTest, X509CsrFromAndToString) {
+  static const DataFormat kFormats[] = { DataFormat::PEM, DataFormat::DER };
+
+  PrivateKey key;
+  ASSERT_OK(GeneratePrivateKey(1024, &key));
+  CertRequestGenerator gen(PrepareConfig(
+      "4C931ADC-3945-4E05-8DB2-447327BF8F62", {"localhost"}));
+  ASSERT_OK(gen.Init());
+  CertSignRequest req_ref;
+  ASSERT_OK(gen.GenerateRequest(key, &req_ref));
+
+  for (auto format : kFormats) {
+    SCOPED_TRACE(Substitute("X509 CSR format: $0", DataFormatToString(format)));
+    string str_req_ref;
+    ASSERT_OK(req_ref.ToString(&str_req_ref, format));
+    CertSignRequest req;
+    ASSERT_OK(req.FromString(str_req_ref, format));
+    string str_req;
+    ASSERT_OK(req.ToString(&str_req, format));
+    ASSERT_EQ(str_req_ref, str_req);
+  }
+}
+
+// Check the transformation chains for X509 certs:
+//   internal -> PEM -> internal -> PEM
+//   internal -> DER -> internal -> DER
+TEST_F(CertManagementTest, X509FromAndToString) {
+  static const DataFormat kFormats[] = { DataFormat::PEM, DataFormat::DER };
+
+  PrivateKey key;
+  ASSERT_OK(GeneratePrivateKey(1024, &key));
+  CertRequestGenerator gen(PrepareConfig(
+      "86F676E9-4E77-4DDC-B15C-596E74B03D90", {"localhost"}));
+  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_));
+  Cert cert_ref;
+  ASSERT_OK(signer.Sign(req, &cert_ref));
+
+  for (auto format : kFormats) {
+    SCOPED_TRACE(Substitute("X509 format: $0", DataFormatToString(format)));
+    string str_cert_ref;
+    ASSERT_OK(cert_ref.ToString(&str_cert_ref, format));
+    Cert cert;
+    ASSERT_OK(cert.FromString(str_cert_ref, format));
+    string str_cert;
+    ASSERT_OK(cert.ToString(&str_cert, format));
+    ASSERT_EQ(str_cert_ref, str_cert);
+  }
+}
+
+// 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/d91313d6/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 99168aa..8de37dd 100644
--- a/src/kudu/security/ca/cert_management.cc
+++ b/src/kudu/security/ca/cert_management.cc
@@ -36,6 +36,7 @@
 #include <openssl/x509v3.h>
 
 #include "kudu/gutil/strings/substitute.h"
+#include "kudu/security/cert.h"
 #include "kudu/security/openssl_util.h"
 #include "kudu/util/scoped_cleanup.h"
 #include "kudu/util/status.h"
@@ -79,13 +80,13 @@ CertRequestGenerator::~CertRequestGenerator() {
   sk_X509_EXTENSION_pop_free(extensions_, X509_EXTENSION_free);
 }
 
-Status CertRequestGeneratorBase::GenerateRequest(const Key& key,
+Status CertRequestGeneratorBase::GenerateRequest(const PrivateKey& key,
                                                  CertSignRequest* ret) const {
   CHECK(ret);
   CHECK(Initialized());
   auto req = ssl_make_unique(X509_REQ_new());
-  CERT_RET_NOT_OK(X509_REQ_set_pubkey(req.get(), key.GetRawData()),
-                  "error setting X509 public key");
+  OPENSSL_RET_NOT_OK(X509_REQ_set_pubkey(req.get(), key.GetRawData()),
+      "error setting X509 public key");
   X509_NAME* name = X509_REQ_get_subject_name(req.get());
   CHECK(name);
 
@@ -93,9 +94,10 @@ Status CertRequestGeneratorBase::GenerateRequest(const Key& key,
   do { \
     const string& f = (field); \
     if (!f.empty()) { \
-      CERT_RET_NOT_OK(X509_NAME_add_entry_by_txt(name, (code), MBSTRING_ASC,  \
+      OPENSSL_RET_NOT_OK(X509_NAME_add_entry_by_txt(\
+          name, (code), MBSTRING_ASC, \
           reinterpret_cast<const unsigned char*>(f.c_str()), -1, -1, 0), \
-         ("error setting subject " # err_msg)); \
+          ("error setting subject " # err_msg)); \
     } \
   } while (false)
 
@@ -111,8 +113,8 @@ Status CertRequestGeneratorBase::GenerateRequest(const Key& key,
   RETURN_NOT_OK(SetExtensions(req.get()));
 
   // And finally sign the result.
-  CERT_RET_NOT_OK(X509_REQ_sign(req.get(), key.GetRawData(), EVP_sha256()),
-                  "error signing X509 request");
+  OPENSSL_RET_NOT_OK(X509_REQ_sign(req.get(), key.GetRawData(), EVP_sha256()),
+      "error signing X509 request");
   ret->AdoptRawData(req.release());
 
   return Status::OK();
@@ -125,8 +127,8 @@ Status CertRequestGeneratorBase::PushExtension(stack_st_X509_EXTENSION* st,
   if (!ex) {
     return Status::RuntimeError("error configuring extension");
   }
-  CERT_RET_NOT_OK(sk_X509_EXTENSION_push(st, ex.release()),
-                  "error pushing extension into the stack");
+  OPENSSL_RET_NOT_OK(sk_X509_EXTENSION_push(st, ex.release()),
+      "error pushing extension into the stack");
   return Status::OK();
 }
 
@@ -219,8 +221,8 @@ bool CertRequestGenerator::Initialized() const {
 }
 
 Status CertRequestGenerator::SetExtensions(X509_REQ* req) const {
-  CERT_RET_NOT_OK(X509_REQ_add_extensions(req, extensions_),
-                  "error setting X509 request extensions");
+  OPENSSL_RET_NOT_OK(X509_REQ_add_extensions(req, extensions_),
+      "error setting X509 request extensions");
   return Status::OK();
 }
 
@@ -274,8 +276,8 @@ bool CaCertRequestGenerator::Initialized() const {
 }
 
 Status CaCertRequestGenerator::SetExtensions(X509_REQ* req) const {
-  CERT_RET_NOT_OK(X509_REQ_add_extensions(req, extensions_),
-                  "error setting X509 request extensions");
+  OPENSSL_RET_NOT_OK(X509_REQ_add_extensions(req, extensions_),
+      "error setting X509 request extensions");
   return Status::OK();
 }
 
@@ -284,7 +286,7 @@ CertSigner::CertSigner()
 }
 
 Status CertSigner::Init(shared_ptr<Cert> ca_cert,
-                        shared_ptr<Key> ca_private_key) {
+                        shared_ptr<PrivateKey> ca_private_key) {
   CHECK(ca_cert && ca_cert->GetRawData());
   CHECK(ca_private_key && ca_private_key->GetRawData());
   InitializeOpenSSL();
@@ -298,7 +300,7 @@ Status CertSigner::Init(shared_ptr<Cert> ca_cert,
   return Status::OK();
 }
 
-Status CertSigner::InitForSelfSigning(shared_ptr<Key> private_key) {
+Status CertSigner::InitForSelfSigning(shared_ptr<PrivateKey> private_key) {
   CHECK(private_key);
   InitializeOpenSSL();
 
@@ -317,15 +319,14 @@ Status CertSigner::InitFromFiles(const string& ca_cert_path,
   std::lock_guard<simple_spinlock> guard(lock_);
   CHECK(!is_initialized_);
   auto cert = std::make_shared<Cert>();
-  auto key = std::make_shared<Key>();
+  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));
-  CERT_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));
+  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;
@@ -343,7 +344,7 @@ const Cert& CertSigner::ca_cert() const {
   return *CHECK_NOTNULL(ca_cert_.get());
 }
 
-const Key& CertSigner::ca_private_key() const {
+const PrivateKey& CertSigner::ca_private_key() const {
   lock_guard<simple_spinlock> guard(lock_);
   DCHECK(is_initialized_);
   return *ca_private_key_;
@@ -381,7 +382,7 @@ Status CertSigner::CopyExtensions(X509_REQ* req, X509* x) {
         idx = X509_get_ext_by_OBJ(x, obj, -1);
       } while (idx != -1);
     }
-    CERT_RET_NOT_OK(X509_add_ext(x, ext, -1), "error adding extension");
+    OPENSSL_RET_NOT_OK(X509_add_ext(x, ext, -1), "error adding extension");
   }
 
   return Status::OK();
@@ -406,26 +407,26 @@ Status CertSigner::FillCertTemplateFromRequest(X509_REQ* req, X509* tmpl) {
   if (rc == 0) {
     return Status::RuntimeError("CSR signature mismatch");
   }
-  CERT_RET_NOT_OK(X509_set_subject_name(tmpl, X509_REQ_get_subject_name(req)),
-                  "error setting cert subject name");
+  OPENSSL_RET_NOT_OK(X509_set_subject_name(tmpl, X509_REQ_get_subject_name(req)),
+      "error setting cert subject name");
   RETURN_NOT_OK(CopyExtensions(req, tmpl));
-  CERT_RET_NOT_OK(X509_set_pubkey(tmpl, pub_key.get()),
-                  "error setting cert public key");
+  OPENSSL_RET_NOT_OK(X509_set_pubkey(tmpl, pub_key.get()),
+      "error setting cert public key");
   return Status::OK();
 }
 
 Status CertSigner::DigestSign(const EVP_MD* md, EVP_PKEY* pkey, X509* x) {
-  CERT_RET_NOT_OK(X509_sign(x, pkey, md), "error signing certificate");
+  OPENSSL_RET_NOT_OK(X509_sign(x, pkey, md), "error signing certificate");
   return Status::OK();
 }
 
 Status CertSigner::GenerateSerial(c_unique_ptr<ASN1_INTEGER>* ret) {
   auto btmp = ssl_make_unique(BN_new());
-  CERT_RET_NOT_OK(BN_pseudo_rand(btmp.get(), 64, 0, 0),
-                  "error generating random number");
+  OPENSSL_RET_NOT_OK(BN_pseudo_rand(btmp.get(), 64, 0, 0),
+      "error generating random number");
   auto serial = ssl_make_unique(ASN1_INTEGER_new());
-  CERT_RET_IF_NULL(BN_to_ASN1_INTEGER(btmp.get(), serial.get()),
-                   "error converting number into ASN1 representation");
+  OPENSSL_RET_IF_NULL(BN_to_ASN1_INTEGER(btmp.get(), serial.get()),
+      "error converting number into ASN1 representation");
   if (ret) {
     ret->swap(serial);
   }
@@ -444,18 +445,19 @@ Status CertSigner::DoSign(const EVP_MD* digest, int32_t exp_seconds,
   // 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");
+  OPENSSL_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
-  CERT_RET_NOT_OK(X509_set_version(ret, kX509V3), "error setting cert version");
-  CERT_RET_NOT_OK(X509_set_serialNumber(ret, serial.get()),
-                  "error setting cert serial");
-  CERT_RET_IF_NULL(X509_gmtime_adj(X509_get_notBefore(ret), 0L),
-                   "error setting cert validity time");
-  CERT_RET_IF_NULL(X509_gmtime_adj(X509_get_notAfter(ret), exp_seconds),
-                   "error setting cert expiration time");
+  OPENSSL_RET_NOT_OK(X509_set_version(ret, kX509V3),
+      "error setting cert version");
+  OPENSSL_RET_NOT_OK(X509_set_serialNumber(ret, serial.get()),
+      "error setting cert serial");
+  OPENSSL_RET_IF_NULL(X509_gmtime_adj(X509_get_notBefore(ret), 0L),
+      "error setting cert validity time");
+  OPENSSL_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 Status::OK();

http://git-wip-us.apache.org/repos/asf/kudu/blob/d91313d6/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 4a222f4..4974983 100644
--- a/src/kudu/security/ca/cert_management.h
+++ b/src/kudu/security/ca/cert_management.h
@@ -25,6 +25,7 @@
 #include <vector>
 
 #include "kudu/gutil/macros.h"
+#include "kudu/security/crypto.h"
 #include "kudu/security/openssl_util.h"
 #include "kudu/util/locks.h"
 #include "kudu/util/monotime.h"
@@ -35,10 +36,16 @@
 typedef struct asn1_string_st ASN1_INTEGER;
 typedef struct env_md_st EVP_MD;
 typedef struct rsa_st RSA;
+typedef struct x509_st X509;
+typedef struct X509_req_st X509_REQ;
 struct stack_st_X509_EXTENSION; // STACK_OF(X509_EXTENSION)
 
 namespace kudu {
 namespace security {
+
+class Cert;
+class CertSignRequest;
+
 namespace ca {
 
 // Base utility class for issuing X509 CSRs.
@@ -66,7 +73,7 @@ class CertRequestGeneratorBase {
 
   // Generate X509 CSR using the specified key. To obtain the key,
   // call the GeneratePrivateKey() function.
-  Status GenerateRequest(const Key& key, CertSignRequest* ret) const;
+  Status GenerateRequest(const PrivateKey& key, CertSignRequest* ret) const;
 
  protected:
   // Push the specified extension into the stack provided.
@@ -136,7 +143,7 @@ class CertSigner {
   // 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);
+              std::shared_ptr<PrivateKey> ca_private_key);
 
   // Initialize the signer from a CA cert and private key stored
   // on disk.
@@ -146,7 +153,7 @@ class CertSigner {
   //
   // 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);
+  Status InitForSelfSigning(std::shared_ptr<PrivateKey> private_key);
 
   // Set the expiration interval for certs signed by this signer.
   // This may be changed at any point.
@@ -158,7 +165,7 @@ class CertSigner {
   bool Initialized() const;
 
   const Cert& ca_cert() const;
-  const Key& ca_private_key() const;
+  const PrivateKey& ca_private_key() const;
 
   Status Sign(const CertSignRequest& req, Cert* ret) const;
 
@@ -181,7 +188,7 @@ class CertSigner {
 
   // 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_;
+  std::shared_ptr<PrivateKey> ca_private_key_;
 
   DISALLOW_COPY_AND_ASSIGN(CertSigner);
 };

http://git-wip-us.apache.org/repos/asf/kudu/blob/d91313d6/src/kudu/security/cert.cc
----------------------------------------------------------------------
diff --git a/src/kudu/security/cert.cc b/src/kudu/security/cert.cc
new file mode 100644
index 0000000..b141ad1
--- /dev/null
+++ b/src/kudu/security/cert.cc
@@ -0,0 +1,74 @@
+// 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/security/cert.h"
+
+#include <string>
+
+#include <openssl/pem.h>
+#include <openssl/x509.h>
+
+#include "kudu/security/openssl_util.h"
+#include "kudu/security/openssl_util_bio.h"
+#include "kudu/util/status.h"
+
+using std::string;
+
+namespace kudu {
+namespace security {
+
+template<> struct SslTypeTraits<X509> {
+  static constexpr auto free = &X509_free;
+  static constexpr auto read_pem = &PEM_read_bio_X509;
+  static constexpr auto read_der = &d2i_X509_bio;
+  static constexpr auto write_pem = &PEM_write_bio_X509;
+  static constexpr auto write_der = &i2d_X509_bio;
+};
+template<> struct SslTypeTraits<X509_REQ> {
+  static constexpr auto free = &X509_REQ_free;
+  static constexpr auto read_pem = &PEM_read_bio_X509_REQ;
+  static constexpr auto read_der = &d2i_X509_REQ_bio;
+  static constexpr auto write_pem = &PEM_write_bio_X509_REQ;
+  static constexpr auto write_der = &i2d_X509_REQ_bio;
+};
+
+Status Cert::FromString(const std::string& data, DataFormat format) {
+  return ::kudu::security::FromString(data, format, &data_);
+}
+
+Status Cert::ToString(std::string* data, DataFormat format) const {
+  return ::kudu::security::ToString(data, format, data_.get());
+}
+
+Status Cert::FromFile(const std::string& fpath, DataFormat format) {
+  return ::kudu::security::FromFile(fpath, format, &data_);
+}
+
+Status CertSignRequest::FromString(const std::string& data, DataFormat format) {
+  return ::kudu::security::FromString(data, format, &data_);
+}
+
+Status CertSignRequest::ToString(std::string* data, DataFormat format) const {
+  return ::kudu::security::ToString(data, format, data_.get());
+}
+
+Status CertSignRequest::FromFile(const std::string& fpath, DataFormat format) {
+  return ::kudu::security::FromFile(fpath, format, &data_);
+}
+
+} // namespace security
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/d91313d6/src/kudu/security/cert.h
----------------------------------------------------------------------
diff --git a/src/kudu/security/cert.h b/src/kudu/security/cert.h
new file mode 100644
index 0000000..536d7e9
--- /dev/null
+++ b/src/kudu/security/cert.h
@@ -0,0 +1,49 @@
+// 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 <string>
+
+#include "kudu/security/openssl_util.h"
+
+// Forward declarations for the OpenSSL typedefs.
+typedef struct x509_st X509;
+typedef struct X509_req_st X509_REQ;
+
+namespace kudu {
+
+class Status;
+
+namespace security {
+
+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);
+};
+
+class CertSignRequest : public RawDataWrapper<X509_REQ> {
+ 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);
+};
+
+} // namespace security
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/d91313d6/src/kudu/security/crypto.cc
----------------------------------------------------------------------
diff --git a/src/kudu/security/crypto.cc b/src/kudu/security/crypto.cc
new file mode 100644
index 0000000..856ddda
--- /dev/null
+++ b/src/kudu/security/crypto.cc
@@ -0,0 +1,164 @@
+// 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/security/crypto.h"
+
+#include <cstdio>
+#include <cstdlib>
+#include <string>
+
+#include <glog/logging.h>
+#include <openssl/bio.h>
+#include <openssl/evp.h>
+#include <openssl/pem.h>
+
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/security/openssl_util.h"
+#include "kudu/security/openssl_util_bio.h"
+#include "kudu/util/status.h"
+
+using std::string;
+using strings::Substitute;
+
+namespace kudu {
+namespace security {
+
+namespace {
+
+// Writing the private key from an EVP_PKEY has a different
+// signature than the rest of the write functions, so we
+// have to provide this wrapper.
+int PemWritePrivateKey(BIO* bio, EVP_PKEY* key) {
+  auto rsa = ssl_make_unique(EVP_PKEY_get1_RSA(key));
+  return PEM_write_bio_RSAPrivateKey(
+      bio, rsa.get(), nullptr, nullptr, 0, nullptr, nullptr);
+}
+
+int PemWritePublicKey(BIO* bio, EVP_PKEY* key) {
+  auto rsa = ssl_make_unique(EVP_PKEY_get1_RSA(key));
+  return PEM_write_bio_RSA_PUBKEY(bio, rsa.get());
+}
+
+int DerWritePublicKey(BIO* bio, EVP_PKEY* key) {
+  auto rsa = ssl_make_unique(EVP_PKEY_get1_RSA(key));
+  return i2d_RSA_PUBKEY_bio(bio, rsa.get());
+}
+
+} // anonymous namespace
+
+template<> struct SslTypeTraits<BIGNUM> {
+  static constexpr auto free = &BN_free;
+};
+template<> struct SslTypeTraits<EVP_PKEY> {
+  static constexpr auto free = &EVP_PKEY_free;
+};
+struct RsaPrivateKeyTraits : public SslTypeTraits<EVP_PKEY> {
+  static constexpr auto read_pem = &PEM_read_bio_PrivateKey;
+  static constexpr auto read_der = &d2i_PrivateKey_bio;
+  static constexpr auto write_pem = &PemWritePrivateKey;
+  static constexpr auto write_der = &i2d_PrivateKey_bio;
+};
+struct RsaPublicKeyTraits : public SslTypeTraits<EVP_PKEY> {
+  static constexpr auto read_pem = &PEM_read_bio_PUBKEY;
+  static constexpr auto read_der = &d2i_PUBKEY_bio;
+  static constexpr auto write_pem = &PemWritePublicKey;
+  static constexpr auto write_der = &DerWritePublicKey;
+};
+template<> struct SslTypeTraits<RSA> {
+  static constexpr auto free = &RSA_free;
+};
+template<> struct SslTypeTraits<void> {
+    static constexpr auto free = &CRYPTO_free;
+};
+
+
+Status PublicKey::FromString(const std::string& data, DataFormat format) {
+  return ::kudu::security::FromString<RawDataType, RsaPublicKeyTraits>(
+      data, format, &data_);
+}
+
+Status PublicKey::ToString(std::string* data, DataFormat format) const {
+  return ::kudu::security::ToString<RawDataType, RsaPublicKeyTraits>(
+      data, format, data_.get());
+}
+
+Status PublicKey::FromFile(const std::string& fpath, DataFormat format) {
+  return ::kudu::security::FromFile<RawDataType, RsaPublicKeyTraits>(
+      fpath, format, &data_);
+}
+
+Status PublicKey::FromBIO(BIO* bio, DataFormat format) {
+  return ::kudu::security::FromBIO<RawDataType, RsaPublicKeyTraits>(
+      bio, format, &data_);
+}
+
+Status PrivateKey::FromString(const std::string& data, DataFormat format) {
+  return ::kudu::security::FromString<RawDataType, RsaPrivateKeyTraits>(
+      data, format, &data_);
+}
+
+Status PrivateKey::ToString(std::string* data, DataFormat format) const {
+  return ::kudu::security::ToString<RawDataType, RsaPrivateKeyTraits>(
+      data, format, data_.get());
+}
+
+Status PrivateKey::FromFile(const std::string& fpath, DataFormat format) {
+  return ::kudu::security::FromFile<RawDataType, RsaPrivateKeyTraits>(
+      fpath, format, &data_);
+}
+
+// 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) {
+  CHECK(public_key);
+  auto rsa = ssl_make_unique(EVP_PKEY_get1_RSA(CHECK_NOTNULL(data_.get())));
+  if (PREDICT_FALSE(!rsa)) {
+    return Status::RuntimeError(GetOpenSSLErrors());
+  }
+  auto tmp = ssl_make_unique(BIO_new(BIO_s_mem()));
+  CHECK(tmp);
+  // Export public key in DER format into the temporary buffer.
+  OPENSSL_RET_NOT_OK(i2d_RSA_PUBKEY_bio(tmp.get(), rsa.get()),
+      "error extracting public RSA key");
+  // Read the public key into the result placeholder.
+  RETURN_NOT_OK(public_key->FromBIO(tmp.get(), DataFormat::DER));
+
+  return Status::OK();
+}
+
+Status GeneratePrivateKey(int num_bits, PrivateKey* ret) {
+  CHECK(ret);
+  InitializeOpenSSL();
+  auto key = ssl_make_unique(EVP_PKEY_new());
+  {
+    auto bn = ssl_make_unique(BN_new());
+    OPENSSL_CHECK_OK(BN_set_word(bn.get(), RSA_F4));
+    auto rsa = ssl_make_unique(RSA_new());
+    OPENSSL_RET_NOT_OK(
+        RSA_generate_key_ex(rsa.get(), num_bits, bn.get(), nullptr),
+        "error generating RSA key");
+    OPENSSL_RET_NOT_OK(EVP_PKEY_set1_RSA(key.get(), rsa.get()),
+        "error assigning RSA key");
+  }
+  ret->AdoptRawData(key.release());
+
+  return Status::OK();
+}
+
+} // namespace security
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/d91313d6/src/kudu/security/crypto.h
----------------------------------------------------------------------
diff --git a/src/kudu/security/crypto.h b/src/kudu/security/crypto.h
new file mode 100644
index 0000000..1da5de3
--- /dev/null
+++ b/src/kudu/security/crypto.h
@@ -0,0 +1,66 @@
+// 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 <string>
+
+#include "kudu/security/openssl_util.h"
+
+// Forward declarations for the OpenSSL typedefs.
+typedef struct rsa_st RSA;
+typedef struct bio_st BIO;
+
+namespace kudu {
+
+class Status;
+
+namespace security {
+
+// A class with generic public key interface, but actually it represents
+// an RSA key.
+class PublicKey : public RawDataWrapper<EVP_PKEY> {
+ public:
+  ~PublicKey() {}
+
+  Status FromString(const std::string& data, DataFormat format);
+  Status ToString(std::string* data, DataFormat format) const;
+  Status FromFile(const std::string& fpath, DataFormat format);
+
+  Status FromBIO(BIO* bio, DataFormat format);
+};
+
+// A class with generic private key interface, but actually it represents
+// an RSA private key. It's important to have PrivateKey and PublicKey
+// be different types to avoid accidental leakage of private keys.
+class PrivateKey : public RawDataWrapper<EVP_PKEY> {
+ public:
+  ~PrivateKey() {}
+
+  Status FromString(const std::string& data, DataFormat format);
+  Status ToString(std::string* data, DataFormat format) const;
+  Status FromFile(const std::string& fpath, DataFormat format);
+
+  // Output the public part of the keypair into the specified placeholder.
+  Status GetPublicKey(PublicKey* public_key);
+};
+
+// Utility method to generate private keys.
+Status GeneratePrivateKey(int num_bits, PrivateKey* ret);
+
+} // namespace security
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/d91313d6/src/kudu/security/openssl_util.cc
----------------------------------------------------------------------
diff --git a/src/kudu/security/openssl_util.cc b/src/kudu/security/openssl_util.cc
index be6a821..c32cb72 100644
--- a/src/kudu/security/openssl_util.cc
+++ b/src/kudu/security/openssl_util.cc
@@ -135,92 +135,6 @@ string GetSSLErrorDescription(int error_code) {
   }
 }
 
-namespace {
-
-// Writing the private key from an EVP_PKEY has a different
-// signature than the rest of the write functions, so we
-// have to provide this wrapper.
-int PemWritePrivateKey(BIO* bio, EVP_PKEY* key) {
-  auto rsa = ssl_make_unique(EVP_PKEY_get1_RSA(key));
-  return PEM_write_bio_RSAPrivateKey(
-      bio, rsa.get(), nullptr, nullptr, 0, nullptr, nullptr);
-}
-
-} // anonymous namespace
-
-template<> struct SslTypeTraits<BIO> {
-  static constexpr auto free = &BIO_free;
-};
-template<> struct SslTypeTraits<BIGNUM> {
-  static constexpr auto free = &BN_free;
-};
-template<> struct SslTypeTraits<EVP_PKEY> {
-  static constexpr auto free = &EVP_PKEY_free;
-  static constexpr auto read_pem = &PEM_read_bio_PrivateKey;
-  static constexpr auto read_der = &d2i_PrivateKey_bio;
-  static constexpr auto write_pem = &PemWritePrivateKey;
-  static constexpr auto write_der = &i2d_PrivateKey_bio;
-};
-template<> struct SslTypeTraits<RSA> {
-  static constexpr auto free = &RSA_free;
-};
-template<> struct SslTypeTraits<X509> {
-  static constexpr auto free = &X509_free;
-  static constexpr auto read_pem = &PEM_read_bio_X509;
-  static constexpr auto read_der = &d2i_X509_bio;
-  static constexpr auto write_pem = &PEM_write_bio_X509;
-  static constexpr auto write_der = &i2d_X509_bio;
-};
-template<> struct SslTypeTraits<X509_REQ> {
-  static constexpr auto free = &X509_REQ_free;
-  static constexpr auto read_pem = &PEM_read_bio_X509_REQ;
-  static constexpr auto read_der = &d2i_X509_REQ_bio;
-  static constexpr auto write_pem = &PEM_write_bio_X509_REQ;
-  static constexpr auto write_der = &i2d_X509_REQ_bio;
-};
-
-namespace {
-
-template<class TYPE>
-Status ToBIO(BIO* bio, DataFormat format, TYPE* obj) {
-  using Traits = SslTypeTraits<TYPE>;
-  CHECK(bio);
-  CHECK(obj);
-  switch (format) {
-    case DataFormat::DER:
-      CERT_RET_NOT_OK(Traits::write_der(bio, obj),
-          "error exporting data in DER format");
-      break;
-    case DataFormat::PEM:
-      CERT_RET_NOT_OK(Traits::write_pem(bio, obj),
-          "error exporting data in PEM format");
-      break;
-  }
-  CERT_RET_NOT_OK(BIO_flush(bio), "error flushing BIO");
-  return Status::OK();
-}
-
-template<class TYPE>
-Status FromBIO(BIO* bio, DataFormat format, c_unique_ptr<TYPE>* ret) {
-  using Traits = SslTypeTraits<TYPE>;
-  CHECK(bio);
-  switch (format) {
-    case DataFormat::DER:
-      *ret = ssl_make_unique(Traits::read_der(bio, nullptr));
-      break;
-    case DataFormat::PEM:
-      *ret = ssl_make_unique(Traits::read_pem(bio, nullptr, nullptr, nullptr));
-      break;
-  }
-  if (PREDICT_FALSE(!*ret)) {
-    return Status::RuntimeError(GetOpenSSLErrors());
-  }
-  return Status::OK();
-}
-
-} // anonymous namespace
-
-
 const string& DataFormatToString(DataFormat fmt) {
   static const string kStrFormatUnknown = "UNKNOWN";
   static const string kStrFormatDer = "DER";
@@ -235,105 +149,5 @@ const string& DataFormatToString(DataFormat fmt) {
   }
 }
 
-Status BasicWrapper::FromFile(const string& fpath, DataFormat format) {
-  auto bio = ssl_make_unique(BIO_new(BIO_s_file()));
-  CERT_RET_NOT_OK(BIO_read_filename(bio.get(), fpath.c_str()),
-                  Substitute("$0: could not read from file", fpath));
-  RETURN_NOT_OK_PREPEND(FromBIO(bio.get(), format),
-                        Substitute("$0: unable to load data key from file",
-                                   fpath));
-  return Status::OK();
-}
-
-Status BasicWrapper::FromString(const string& data, DataFormat format) {
-  const void* mdata = reinterpret_cast<const void*>(data.data());
-  auto bio = ssl_make_unique(BIO_new_mem_buf(
-#if OPENSSL_VERSION_NUMBER < 0x10002000L
-      const_cast<void*>(mdata),
-#else
-      mdata,
-#endif
-      data.size()));
-  RETURN_NOT_OK_PREPEND(FromBIO(bio.get(), format),
-                        "unable to load data from memory");
-  return Status::OK();
-}
-
-Status BasicWrapper::ToString(std::string* data, DataFormat format) const {
-  CHECK(data);
-  auto bio = ssl_make_unique(BIO_new(BIO_s_mem()));
-  RETURN_NOT_OK_PREPEND(ToBIO(bio.get(), format), "error serializing data");
-  BUF_MEM* membuf;
-  CERT_CHECK_OK(BIO_get_mem_ptr(bio.get(), &membuf));
-  data->assign(membuf->data, membuf->length);
-  return Status::OK();
-}
-
-void Key::AdoptRawData(RawDataType* data) {
-  data_ = ssl_make_unique(data);
-}
-
-Status Key::FromBIO(BIO* bio, DataFormat format) {
-  RETURN_NOT_OK_PREPEND(::kudu::security::FromBIO(bio, format, &data_),
-      "unable to read private key");
-  return Status::OK();
-}
-
-Status Key::ToBIO(BIO* bio, DataFormat format) const {
-  RETURN_NOT_OK_PREPEND(::kudu::security::ToBIO(bio, format, data_.get()),
-      "could not export cert");
-  return Status::OK();
-}
-
-void Cert::AdoptRawData(RawDataType* data) {
-  data_ = ssl_make_unique(data);
-}
-
-Status Cert::FromBIO(BIO* bio, DataFormat format) {
-  RETURN_NOT_OK_PREPEND(::kudu::security::FromBIO(bio, format, &data_),
-      "could not read cert");
-  return Status::OK();
-}
-
-Status Cert::ToBIO(BIO* bio, DataFormat format) const {
-  RETURN_NOT_OK_PREPEND(::kudu::security::ToBIO(bio, format, data_.get()),
-      "could not export cert");
-  return Status::OK();
-}
-
-void CertSignRequest::AdoptRawData(RawDataType* data) {
-  data_ = ssl_make_unique(data);
-}
-
-Status CertSignRequest::FromBIO(BIO* bio, DataFormat format) {
-  RETURN_NOT_OK_PREPEND(::kudu::security::FromBIO(bio, format, &data_),
-      "could not read X509 CSR");
-  return Status::OK();
-}
-
-Status CertSignRequest::ToBIO(BIO* bio, DataFormat format) const {
-  RETURN_NOT_OK_PREPEND(::kudu::security::ToBIO(bio, format, data_.get()),
-      "could not export X509 CSR");
-  return Status::OK();
-}
-
-Status GeneratePrivateKey(int num_bits, Key* ret) {
-  CHECK(ret);
-  InitializeOpenSSL();
-  auto key = ssl_make_unique(EVP_PKEY_new());
-  {
-    auto bn = ssl_make_unique(BN_new());
-    CERT_CHECK_OK(BN_set_word(bn.get(), RSA_F4));
-    auto rsa = ssl_make_unique(RSA_new());
-    CERT_RET_NOT_OK(RSA_generate_key_ex(rsa.get(), num_bits, bn.get(), nullptr),
-                    "error generating RSA key");
-    CERT_RET_NOT_OK(EVP_PKEY_set1_RSA(key.get(), rsa.get()),
-                    "error assigning RSA key");
-  }
-  ret->AdoptRawData(key.release());
-
-  return Status::OK();
-}
-
 } // namespace security
 } // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/d91313d6/src/kudu/security/openssl_util.h
----------------------------------------------------------------------
diff --git a/src/kudu/security/openssl_util.h b/src/kudu/security/openssl_util.h
index ab3567c..1691727 100644
--- a/src/kudu/security/openssl_util.h
+++ b/src/kudu/security/openssl_util.h
@@ -21,31 +21,27 @@
 #include <memory>
 #include <string>
 
-#include <glog/logging.h>
-
-#include "kudu/util/status.h"
 #include "kudu/gutil/strings/substitute.h"
+#include "kudu/util/status.h"
 
 // Forward declarations for the OpenSSL typedefs.
 typedef struct bio_st BIO;
 typedef struct evp_pkey_st EVP_PKEY;
 typedef struct ssl_ctx_st SSL_CTX;
 typedef struct ssl_st SSL;
-typedef struct x509_st X509;
-typedef struct X509_req_st X509_REQ;
 
-#define CERT_CHECK_OK(call) \
+#define OPENSSL_CHECK_OK(call) \
   CHECK_GT((call), 0)
 
-#define CERT_RET_NOT_OK(call, msg) \
+#define OPENSSL_RET_NOT_OK(call, msg) \
   if ((call) <= 0) { \
-    return Status::RuntimeError(Substitute("$0: $1", \
+    return Status::RuntimeError(::strings::Substitute("$0: $1", \
         (msg), GetOpenSSLErrors())); \
   }
 
-#define CERT_RET_IF_NULL(call, msg) \
+#define OPENSSL_RET_IF_NULL(call, msg) \
   if ((call) == nullptr) { \
-    return Status::RuntimeError(Substitute("$0: $1", \
+    return Status::RuntimeError(::strings::Substitute("$0: $1", \
         (msg), GetOpenSSLErrors())); \
   }
 
@@ -74,103 +70,48 @@ std::string GetOpenSSLErrors();
 // See man(3) SSL_get_error for more discussion.
 std::string GetSSLErrorDescription(int error_code);
 
+
 // A generic wrapper for OpenSSL structures.
 template <typename T>
 using c_unique_ptr = std::unique_ptr<T, std::function<void(T*)>>;
 
 // For each SSL type, the Traits class provides the important OpenSSL
 // API functions.
-template<class SSL_TYPE>
+template<typename SSL_TYPE>
 struct SslTypeTraits {};
 
-template<class SSL_TYPE>
+template<typename SSL_TYPE, typename Traits = SslTypeTraits<SSL_TYPE>>
 c_unique_ptr<SSL_TYPE> ssl_make_unique(SSL_TYPE* d) {
-  return {d, SslTypeTraits<SSL_TYPE>::free};
+  return {d, Traits::free};
 }
 
-// Acceptable formats for X509 certificates, X509 CSRs, and private keys.
+// Acceptable formats for keys, X509 certificates and X509 CSRs.
 enum class DataFormat {
-  DER = 0,    // DER/ASN1 format (binary)
-  PEM = 1,    // PEM format (ASCII)
+  DER = 0,    // DER/ASN1 format (binary): for representing object on the wire
+  PEM = 1,    // PEM format (ASCII): for storing on filesystem, printing, etc.
 };
 
 // Data format representation as a string.
 const std::string& DataFormatToString(DataFormat fmt);
 
-// Basic wrapper for objects of xxx_st type in the OpenSSL crypto library.
-class BasicWrapper {
- public:
-  virtual ~BasicWrapper() = default;
-
-  Status FromFile(const std::string& fpath, DataFormat format);
-  Status FromString(const std::string& data, DataFormat format);
-
-  Status ToString(std::string* data, DataFormat format) const;
-
- protected:
-  virtual Status FromBIO(BIO* bio, DataFormat format) = 0;
-  virtual Status ToBIO(BIO* bio, DataFormat format) const = 0;
-};
-
-// A wrapper for a private key.
-class Key : public BasicWrapper {
- public:
-  typedef EVP_PKEY RawDataType;
-
-  RawDataType* GetRawData() const {
-    return data_.get();
-  }
-
-  void AdoptRawData(RawDataType* data);
-
- protected:
-  Status FromBIO(BIO* bio, DataFormat format) override;
-  Status ToBIO(BIO* bio, DataFormat format) const override;
-
- private:
-  c_unique_ptr<RawDataType> data_;
-};
-
-// A wrapper for a X509 certificate.
-class Cert : public BasicWrapper {
+// Template wrapper for dynamically allocated entities with custom deleter.
+// Mostly, using it for xxx_st types from the OpenSSL crypto library.
+template<typename Type>
+class RawDataWrapper {
  public:
-  typedef X509 RawDataType;
+  typedef Type RawDataType;
 
   RawDataType* GetRawData() const {
     return data_.get();
   }
 
-  void AdoptRawData(RawDataType* data);
-
- protected:
-  Status FromBIO(BIO* bio, DataFormat format) override;
-  Status ToBIO(BIO* bio, DataFormat format) const override;
-
- private:
-  c_unique_ptr<RawDataType> data_;
-};
-
-// A wrapper for a X509 CSR (certificate signing request).
-class CertSignRequest : public BasicWrapper {
- public:
-  typedef X509_REQ RawDataType;
-
-  RawDataType* GetRawData() const {
-    return data_.get();
+  void AdoptRawData(RawDataType* d) {
+    data_ = ssl_make_unique(d);
   }
 
-  void AdoptRawData(RawDataType* data);
-
  protected:
-  Status FromBIO(BIO* bio, DataFormat format) override;
-  Status ToBIO(BIO* bio, DataFormat format) const override;
-
- private:
   c_unique_ptr<RawDataType> data_;
 };
 
-// Utility method to generate private RSA keys.
-Status GeneratePrivateKey(int num_bits, Key* ret);
-
 } // namespace security
 } // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/d91313d6/src/kudu/security/openssl_util_bio.h
----------------------------------------------------------------------
diff --git a/src/kudu/security/openssl_util_bio.h b/src/kudu/security/openssl_util_bio.h
new file mode 100644
index 0000000..6ab6a55
--- /dev/null
+++ b/src/kudu/security/openssl_util_bio.h
@@ -0,0 +1,113 @@
+// 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 "kudu/security/openssl_util.h"
+
+#include <string>
+
+#include <glog/logging.h>
+#include <openssl/bio.h>
+#include <openssl/buffer.h>
+
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/util/status.h"
+
+namespace kudu {
+namespace security {
+
+template<> struct SslTypeTraits<BIO> {
+  static constexpr auto free = &BIO_free;
+};
+
+template<typename TYPE, typename Traits = SslTypeTraits<TYPE>>
+Status ToBIO(BIO* bio, DataFormat format, TYPE* obj) {
+  CHECK(bio);
+  CHECK(obj);
+  switch (format) {
+    case DataFormat::DER:
+      OPENSSL_RET_NOT_OK(Traits::write_der(bio, obj),
+          "error exporting data in DER format");
+      break;
+    case DataFormat::PEM:
+      OPENSSL_RET_NOT_OK(Traits::write_pem(bio, obj),
+          "error exporting data in PEM format");
+      break;
+  }
+  OPENSSL_RET_NOT_OK(BIO_flush(bio), "error flushing BIO");
+  return Status::OK();
+}
+
+template<typename TYPE, typename Traits = SslTypeTraits<TYPE>>
+Status FromBIO(BIO* bio, DataFormat format, c_unique_ptr<TYPE>* ret) {
+  CHECK(bio);
+  switch (format) {
+    case DataFormat::DER:
+      *ret = ssl_make_unique(Traits::read_der(bio, nullptr));
+      break;
+    case DataFormat::PEM:
+      *ret = ssl_make_unique(Traits::read_pem(bio, nullptr, nullptr, nullptr));
+      break;
+  }
+  if (PREDICT_FALSE(!*ret)) {
+    return Status::RuntimeError(GetOpenSSLErrors());
+  }
+  return Status::OK();
+}
+
+template<typename Type, typename Traits = SslTypeTraits<Type>>
+Status FromString(const string& data, DataFormat format,
+                  c_unique_ptr<Type>* ret) {
+  const void* mdata = reinterpret_cast<const void*>(data.data());
+  auto bio = ssl_make_unique(BIO_new_mem_buf(
+#if OPENSSL_VERSION_NUMBER < 0x10002000L
+      const_cast<void*>(mdata),
+#else
+      mdata,
+#endif
+      data.size()));
+  RETURN_NOT_OK_PREPEND((FromBIO<Type, Traits>(bio.get(), format, ret)),
+                        "unable to load data from memory");
+  return Status::OK();
+}
+
+template<typename Type, typename Traits = SslTypeTraits<Type>>
+Status ToString(std::string* data, DataFormat format, Type* obj) {
+  CHECK(data);
+  auto bio = ssl_make_unique(BIO_new(BIO_s_mem()));
+  RETURN_NOT_OK_PREPEND((ToBIO<Type, Traits>(bio.get(), format, obj)),
+                        "error serializing data");
+  BUF_MEM* membuf;
+  OPENSSL_CHECK_OK(BIO_get_mem_ptr(bio.get(), &membuf));
+  data->assign(membuf->data, membuf->length);
+  return Status::OK();
+}
+
+template<typename Type, typename Traits = SslTypeTraits<Type>>
+Status FromFile(const string& fpath, DataFormat format,
+                c_unique_ptr<Type>* ret) {
+  auto bio = ssl_make_unique(BIO_new(BIO_s_file()));
+  OPENSSL_RET_NOT_OK(BIO_read_filename(bio.get(), fpath.c_str()),
+      strings::Substitute("could not read data from file '$0'", fpath));
+  RETURN_NOT_OK_PREPEND((FromBIO<Type, Traits>(bio.get(), format, ret)),
+      strings::Substitute("unable to load data from file '$0'", fpath));
+  return Status::OK();
+}
+
+} // namespace security
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/d91313d6/src/kudu/security/server_cert_manager.cc
----------------------------------------------------------------------
diff --git a/src/kudu/security/server_cert_manager.cc b/src/kudu/security/server_cert_manager.cc
index 650ae47..e682aa9 100644
--- a/src/kudu/security/server_cert_manager.cc
+++ b/src/kudu/security/server_cert_manager.cc
@@ -25,6 +25,7 @@
 
 #include "kudu/util/flag_tags.h"
 #include "kudu/security/ca/cert_management.h"
+#include "kudu/security/cert.h"
 #include "kudu/security/openssl_util.h"
 
 using std::string;
@@ -51,8 +52,9 @@ Status ServerCertManager::Init() {
   MutexLock lock(lock_);
   CHECK(!key_);
 
-  unique_ptr<Key> key(new Key());
-  RETURN_NOT_OK_PREPEND(GeneratePrivateKey(FLAGS_server_rsa_key_length_bits, key.get()),
+  unique_ptr<PrivateKey> key(new PrivateKey());
+  RETURN_NOT_OK_PREPEND(GeneratePrivateKey(FLAGS_server_rsa_key_length_bits,
+                        key.get()),
                         "could not generate private key");
 
   // TODO(aserbin): do these fields actually have to be set?
@@ -74,7 +76,7 @@ Status ServerCertManager::Init() {
   RETURN_NOT_OK_PREPEND(gen.GenerateRequest(*key, &csr), "could not generate CSR");
   RETURN_NOT_OK_PREPEND(csr.ToString(&csr_der_, DataFormat::DER),
                         "unable to output DER format CSR");
-  key_.swap(key);
+  key_ = std::move(key);
   return Status::OK();
 }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/d91313d6/src/kudu/security/server_cert_manager.h
----------------------------------------------------------------------
diff --git a/src/kudu/security/server_cert_manager.h b/src/kudu/security/server_cert_manager.h
index 916bd2d..0a6a26c 100644
--- a/src/kudu/security/server_cert_manager.h
+++ b/src/kudu/security/server_cert_manager.h
@@ -28,9 +28,9 @@
 namespace kudu {
 namespace security {
 
-class Key;
 class Cert;
 class CertSignRequest;
+class PrivateKey;
 
 // Manages the X509 certificate used by a server when the built-in
 // PKI infrastructure is enabled.
@@ -85,7 +85,7 @@ class ServerCertManager {
   std::unique_ptr<Cert> signed_cert_;
 
   // The keypair associated with this server's cert.
-  std::unique_ptr<Key> key_;
+  std::unique_ptr<PrivateKey> key_;
 
   DISALLOW_COPY_AND_ASSIGN(ServerCertManager);
 };

http://git-wip-us.apache.org/repos/asf/kudu/blob/d91313d6/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
deleted file mode 100644
index 857ba9b..0000000
--- a/src/kudu/security/test/cert_management-test.cc
+++ /dev/null
@@ -1,493 +0,0 @@
-// 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/security/ca/cert_management.h"
-
-#include <functional>
-#include <mutex>
-#include <thread>
-#include <utility>
-#include <vector>
-
-#include "kudu/gutil/strings/strip.h"
-#include "kudu/gutil/strings/substitute.h"
-#include "kudu/gutil/strings/util.h"
-#include "kudu/security/openssl_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::string;
-using std::vector;
-using std::thread;
-using strings::Substitute;
-
-namespace kudu {
-namespace security {
-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_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_, kCaExpiredCert, ca_exp_cert_file_));
-    ASSERT_OK(WriteStringToFile(env_, kCaExpiredPrivateKey,
-        ca_exp_private_key_file_));
-  }
-
- 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 = {},
-      const vector<string>& ips = {}) const {
-    const ::testing::TestInfo* const test_info =
-        ::testing::UnitTest::GetInstance()->current_test_info();
-    const string comment = string(test_info->test_case_name()) + "." +
-      test_info->name();
-    const CertRequestGenerator::Config config = {
-      "US",               // country
-      "CA",               // state
-      "San Francisco",    // locality
-      "ASF",              // org
-      "The Kudu Project", // unit
-      uuid,               // uuid
-      comment,            // comment
-      hostnames,          // hostnames
-      ips,                // ips
-    };
-    return config;
-  }
-
-  // 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;
-          Key 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();
-    }
-  }
-
-  const string pem_dir_;
-  const string ca_cert_file_;
-  const string ca_private_key_file_;
-
-  const string ca_exp_cert_file_;
-  const string ca_exp_private_key_file_;
-};
-
-// Check input/output of the keys in PEM format.
-TEST_F(CertManagementTest, KeyInputOutputPEM) {
-  Key key;
-  ASSERT_OK(key.FromFile(ca_private_key_file_, DataFormat::PEM));
-  string key_str;
-  key.ToString(&key_str, DataFormat::PEM);
-  RemoveExtraWhitespace(&key_str);
-
-  string ca_input_key(kCaPrivateKey);
-  RemoveExtraWhitespace(&ca_input_key);
-  EXPECT_EQ(ca_input_key, key_str);
-}
-
-// 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));
-  string cert_str;
-  cert.ToString(&cert_str, DataFormat::PEM);
-  RemoveExtraWhitespace(&cert_str);
-
-  string ca_input_cert(kCaCert);
-  RemoveExtraWhitespace(&ca_input_cert);
-  EXPECT_EQ(ca_input_cert, cert_str);
-}
-
-// Check for basic SAN-related constraints while initializing
-// CertRequestGenerator objects.
-TEST_F(CertManagementTest, RequestGeneratorSanConstraints) {
-  const string kEntityUUID = "D94FBF10-6F40-4F9F-BC82-F96A1C4F2CFB";
-
-  // No hostnames, nor IP addresses are given to populate X509v3 SAN extension.
-  {
-    const CertRequestGenerator::Config gen_config = PrepareConfig(kEntityUUID);
-    CertRequestGenerator gen(gen_config);
-    const Status s = gen.Init();
-    const string err_msg = s.ToString();
-    ASSERT_TRUE(s.IsInvalidArgument()) << err_msg;
-    ASSERT_STR_CONTAINS(err_msg, "SAN: missing DNS names and IP addresses");
-  }
-
-  // An empty hostname
-  {
-    const CertRequestGenerator::Config gen_config =
-        PrepareConfig(kEntityUUID, {"localhost", ""});
-    CertRequestGenerator gen(gen_config);
-    const Status s = gen.Init();
-    const string err_msg = s.ToString();
-    ASSERT_TRUE(s.IsInvalidArgument()) << err_msg;
-    ASSERT_STR_CONTAINS(err_msg, "SAN: an empty hostname");
-  }
-
-  // An empty IP address
-  {
-    const CertRequestGenerator::Config gen_config =
-        PrepareConfig(kEntityUUID, {}, {"127.0.0.1", ""});
-    CertRequestGenerator gen(gen_config);
-    const Status s = gen.Init();
-    const string err_msg = s.ToString();
-    ASSERT_TRUE(s.IsInvalidArgument()) << err_msg;
-    ASSERT_STR_CONTAINS(err_msg, "SAN: an empty IP address");
-  }
-
-  // Missing UUID
-  {
-    const CertRequestGenerator::Config gen_config =
-        PrepareConfig("", {"localhost"});
-    CertRequestGenerator gen(gen_config);
-    const Status s = gen.Init();
-    const string err_msg = s.ToString();
-    ASSERT_TRUE(s.IsInvalidArgument()) << err_msg;
-    ASSERT_STR_CONTAINS(err_msg, "missing end-entity UUID/name");
-  }
-}
-
-// Check for the basic functionality of the CertRequestGenerator class:
-// check it's able to generate keys of expected number of bits and that it
-// reports an error if trying to generate a key of unsupported number of bits.
-TEST_F(CertManagementTest, RequestGeneratorBasics) {
-  const CertRequestGenerator::Config gen_config =
-      PrepareConfig("702C1C5E-CF02-4EDC-8883-07ECDEC8CE97", {"localhost"});
-
-  Key key;
-  ASSERT_OK(GeneratePrivateKey(1024, &key));
-  ASSERT_OK(GeneratePrivateKey(2048, &key));
-  CertRequestGenerator gen(gen_config);
-  ASSERT_OK(gen.Init());
-  string key_str;
-  key.ToString(&key_str, DataFormat::PEM);
-  // Check for non-supported number of bits for the key.
-  Status s = GeneratePrivateKey(7, &key);
-  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) {
-  {
-    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");
-  }
-  {
-    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");
-  }
-}
-
-// Check how CertSigner behaves if given expired CA certificate
-// and corresponding private key.
-TEST_F(CertManagementTest, SignerInitWithExpiredCert) {
-  const CertRequestGenerator::Config gen_config(
-      PrepareConfig("F4466090-BBF8-4042-B72F-BB257500C45A", {"localhost"}));
-  Key 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;
-  // Signer works fine even with expired CA certificate.
-  ASSERT_OK(signer.Sign(req, &cert));
-}
-
-// Generate X509 CSR and issues corresponding certificate: everything is done
-// 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"}));
-  Key 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_));
-  Cert cert;
-  ASSERT_OK(signer.Sign(req, &cert));
-}
-
-// Generate X509 CA CSR and sign the result certificate.
-TEST_F(CertManagementTest, SignCaCert) {
-  const CertRequestGenerator::Config gen_config(
-      PrepareConfig("8C084CF6-A30B-4F5B-9673-A73E62E29A9D"));
-  Key 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_));
-  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
-TEST_F(CertManagementTest, KeyFromAndToString) {
-  static const DataFormat kFormats[] = { DataFormat::PEM, DataFormat::DER };
-  static const uint16_t kKeyBits[] = { 256, 512, 1024, 2048, 3072, 4096 };
-
-  for (auto format : kFormats) {
-    for (auto key_bits : kKeyBits) {
-      SCOPED_TRACE(Substitute("key format: $0, key bits: $1",
-                              format == DataFormat::PEM ? "PEM"
-                                                        : "DER", key_bits));
-      Key key_ref;
-      ASSERT_OK(GeneratePrivateKey(key_bits, &key_ref));
-      string str_key_ref;
-      ASSERT_OK(key_ref.ToString(&str_key_ref, format));
-      Key key;
-      ASSERT_OK(key.FromString(str_key_ref, format));
-      string str_key;
-      ASSERT_OK(key.ToString(&str_key, format));
-      ASSERT_EQ(str_key_ref, str_key);
-    }
-  }
-}
-
-// Check the transformation chains for X509 CSRs:
-//   internal -> PEM -> internal -> PEM
-//   internal -> DER -> internal -> DER
-TEST_F(CertManagementTest, X509CsrFromAndToString) {
-  static const DataFormat kFormats[] = { DataFormat::PEM, DataFormat::DER };
-
-  Key key;
-  ASSERT_OK(GeneratePrivateKey(1024, &key));
-  CertRequestGenerator gen(PrepareConfig(
-      "4C931ADC-3945-4E05-8DB2-447327BF8F62", {"localhost"}));
-  ASSERT_OK(gen.Init());
-  CertSignRequest req_ref;
-  ASSERT_OK(gen.GenerateRequest(key, &req_ref));
-
-  for (auto format : kFormats) {
-    SCOPED_TRACE(Substitute("X509 CSR format: $0", DataFormatToString(format)));
-    string str_req_ref;
-    ASSERT_OK(req_ref.ToString(&str_req_ref, format));
-    CertSignRequest req;
-    ASSERT_OK(req.FromString(str_req_ref, format));
-    string str_req;
-    ASSERT_OK(req.ToString(&str_req, format));
-    ASSERT_EQ(str_req_ref, str_req);
-  }
-}
-
-// Check the transformation chains for X509 certs:
-//   internal -> PEM -> internal -> PEM
-//   internal -> DER -> internal -> DER
-TEST_F(CertManagementTest, X509FromAndToString) {
-  static const DataFormat kFormats[] = { DataFormat::PEM, DataFormat::DER };
-
-  Key key;
-  ASSERT_OK(GeneratePrivateKey(1024, &key));
-  CertRequestGenerator gen(PrepareConfig(
-      "86F676E9-4E77-4DDC-B15C-596E74B03D90", {"localhost"}));
-  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_));
-  Cert cert_ref;
-  ASSERT_OK(signer.Sign(req, &cert_ref));
-
-  for (auto format : kFormats) {
-    SCOPED_TRACE(Substitute("X509 format: $0", DataFormatToString(format)));
-    string str_cert_ref;
-    ASSERT_OK(cert_ref.ToString(&str_cert_ref, format));
-    Cert cert;
-    ASSERT_OK(cert.FromString(str_cert_ref, format));
-    string str_cert;
-    ASSERT_OK(cert.ToString(&str_cert, format));
-    ASSERT_EQ(str_cert_ref, str_cert);
-  }
-}
-
-// 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


[2/3] kudu git commit: [security] method to extract public part of an RSA key

Posted by to...@apache.org.
http://git-wip-us.apache.org/repos/asf/kudu/blob/d91313d6/src/kudu/security/test/test_certs.cc
----------------------------------------------------------------------
diff --git a/src/kudu/security/test/test_certs.cc b/src/kudu/security/test/test_certs.cc
index 1f1e5e5..44d4aef 100644
--- a/src/kudu/security/test/test_certs.cc
+++ b/src/kudu/security/test/test_certs.cc
@@ -157,6 +157,19 @@ gmbcYCewtt7dFP9tvx6k7aUQ6CKzg0GxaIHQecNzjxYrw8sb4Js=
 -----END RSA PRIVATE KEY-----
 )***";
 
+// Corresponding public key for the kCaPrivateKey
+const char kCaPublicKey[] = R"***(
+-----BEGIN PUBLIC KEY-----
+MIIBIjANBgkqhkiG9w0BAQEFAAOCAQ8AMIIBCgKCAQEAp7FdU57RJ817HGXWeowF
+HLwwJhZOXWRtG3lVDr3xpdhw80ELuCcQFb7xhprZ/ceXVh7nXNQac6XgVTocUOIi
+rn3+UoI0bZfdqaViPOQiZefgznqv7msRaz4egdaS6qYxrGeslq0zzo5/QrePlCs5
+R+3+pPlHq9I+cUf+nP7pxwGQAZXRQVV2yFhIaTErOJVN0nZGLyhmDJ65F4bEuRc6
+oofNw5QWKe9Dx97zJpCvWREBo9GjV4Wo5FcAwBTgrK9+JWNkwQaPSnjsh6QkdU9V
+UyikSYwyOHNF9uvGsqsxwgbp7hpwXUoB3jAYj0CkJHd7W9x9uCEM7saFMyzAsKcx
+hQIDAQAB
+-----END PUBLIC KEY-----
+)***";
+
 // See the comment for kCaCert_
 // (but use '-1' as number of days for the certificate expiration).
 const char kCaExpiredCert[] = R"***(
@@ -215,6 +228,19 @@ H/sbP2R+P6RvQceLEEtk6ZZLiuScVmLtVOpUoUZb3Rx6a7GKbec7oQ==
 -----END RSA PRIVATE KEY-----
 )***";
 
+// Corresponding public part of the kCaExpiredPrivateKey
+const char kCaExpiredPublicKey[] = R"***(
+-----BEGIN PUBLIC KEY-----
+MIIBIjANBgkqhkiG9w0BAQEFAAOCAQ8AMIIBCgKCAQEAzqPj5nRm57mr9YtZDvHR
+EuVFHTwPcKzDeff9fnrKKwOJPSF0Bou/BjS1S7yQYAtmT/EMi7qxEWjgrR1qW+mu
+R8QN+zAwNdkdLrFK3SJigQ4a/OeSH86aHXUDekV8mgBgzP90osbHf7AiqrGzkYWq
++ApTO/IgnXgaWbbdt5znGTW5lKQ4O2CYhpcMMC1sBBjW7Qqx+Gi8iXub0zlJ2mVI
+8o+zb9qvSDb8fa0JYxasRDn/nB0wKZC3f/GfRs+lJZUTEy5+eMhVdj1RjVBE+mgW
+7L27On24ViPU7B3DjM0SYnD6ZOUWMH0mtwO8W3OoK8MJhPvFP7Lr5QfSjiBH+ryL
+OwIDAQAB
+-----END PUBLIC KEY-----
+)***";
+
 } // namespace ca
 
 Status CreateTestSSLCerts(const string& dir,

http://git-wip-us.apache.org/repos/asf/kudu/blob/d91313d6/src/kudu/security/test/test_certs.h
----------------------------------------------------------------------
diff --git a/src/kudu/security/test/test_certs.h b/src/kudu/security/test/test_certs.h
index 863c141..f45ea88 100644
--- a/src/kudu/security/test/test_certs.h
+++ b/src/kudu/security/test/test_certs.h
@@ -34,12 +34,16 @@ extern const char kCaCert[];
 // The private key (RSA, 2048 bits) for the certificate above.
 // This is 2048 bit RSA key, in PEM format.
 extern const char kCaPrivateKey[];
+// The public part of the abovementioned private key.
+extern const char kCaPublicKey[];
 
 // Expired root CA certificate (PEM format).
 extern const char kCaExpiredCert[];
 // The private key for the expired CA certificate described above.
 // This is 2048 bit RSA key, in PEM format.
 extern const char kCaExpiredPrivateKey[];
+// The public part of the abovementioned private key.
+extern const char kCaExpiredPublicKey[];
 
 } // namespace ca
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/d91313d6/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 08f32ba..a8f5f6d 100644
--- a/src/kudu/security/tls_handshake-test.cc
+++ b/src/kudu/security/tls_handshake-test.cc
@@ -25,6 +25,8 @@
 #include "kudu/security/tls_context.h"
 #include "kudu/util/test_util.h"
 
+using std::string;
+
 namespace kudu {
 namespace security {
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/d91313d6/src/kudu/security/tls_handshake.h
----------------------------------------------------------------------
diff --git a/src/kudu/security/tls_handshake.h b/src/kudu/security/tls_handshake.h
index a138c76..e7fcdd2 100644
--- a/src/kudu/security/tls_handshake.h
+++ b/src/kudu/security/tls_handshake.h
@@ -20,6 +20,7 @@
 #include <memory>
 #include <string>
 
+#include "kudu/security/crypto.h"
 #include "kudu/security/openssl_util.h"
 #include "kudu/util/net/socket.h"
 #include "kudu/util/status.h"

http://git-wip-us.apache.org/repos/asf/kudu/blob/d91313d6/src/kudu/security/tls_socket.cc
----------------------------------------------------------------------
diff --git a/src/kudu/security/tls_socket.cc b/src/kudu/security/tls_socket.cc
index 20b023e..6728133 100644
--- a/src/kudu/security/tls_socket.cc
+++ b/src/kudu/security/tls_socket.cc
@@ -20,6 +20,7 @@
 #include <openssl/err.h>
 #include <openssl/ssl.h>
 
+#include "kudu/gutil/basictypes.h"
 #include "kudu/security/openssl_util.h"
 
 namespace kudu {

http://git-wip-us.apache.org/repos/asf/kudu/blob/d91313d6/src/kudu/security/token_signer.cc
----------------------------------------------------------------------
diff --git a/src/kudu/security/token_signer.cc b/src/kudu/security/token_signer.cc
index e40c86b..46a8846 100644
--- a/src/kudu/security/token_signer.cc
+++ b/src/kudu/security/token_signer.cc
@@ -19,11 +19,12 @@
 
 #include <map>
 #include <memory>
-#include <string>
+#include <vector>
 
 #include <gflags/gflags.h>
 
 #include "kudu/gutil/walltime.h"
+#include "kudu/security/crypto.h"
 #include "kudu/security/openssl_util.h"
 #include "kudu/security/token.pb.h"
 #include "kudu/security/token_signing_key.h"
@@ -39,8 +40,8 @@ DEFINE_int64(token_signing_key_validity_seconds, 60 * 60 * 24 * 7,
 // TODO(PKI): add flag tags
 
 using std::lock_guard;
-using std::string;
 using std::unique_ptr;
+using std::vector;
 
 namespace kudu {
 namespace security {
@@ -53,14 +54,15 @@ TokenSigner::~TokenSigner() {
 }
 
 Status TokenSigner::RotateSigningKey() {
-  Key key;
-  RETURN_NOT_OK_PREPEND(GeneratePrivateKey(FLAGS_token_signing_key_num_rsa_bits, &key),
-                        "could not generate new RSA token-signing key");
+  unique_ptr<PrivateKey> key(new PrivateKey());
+  RETURN_NOT_OK_PREPEND(
+      GeneratePrivateKey(FLAGS_token_signing_key_num_rsa_bits, key.get()),
+      "could not generate new RSA token-signing key");
   int64_t expire = WallTime_Now() + FLAGS_token_signing_key_validity_seconds;
   lock_guard<RWMutex> l(lock_);
   int64_t seq = next_seq_num_++;
-  unique_ptr<TokenSigningPrivateKey> new_tsk(
-      new TokenSigningPrivateKey(seq, expire, std::move(key)));
+  unique_ptr<TokenSigningPrivateKey> new_tsk(new TokenSigningPrivateKey(
+        seq, expire, unique_ptr<PrivateKey>(key.release())));
   keys_by_seq_[seq] = std::move(new_tsk);
   return Status::OK();
 }

http://git-wip-us.apache.org/repos/asf/kudu/blob/d91313d6/src/kudu/security/token_signing_key.cc
----------------------------------------------------------------------
diff --git a/src/kudu/security/token_signing_key.cc b/src/kudu/security/token_signing_key.cc
index 5971e63..ba84ee6 100644
--- a/src/kudu/security/token_signing_key.cc
+++ b/src/kudu/security/token_signing_key.cc
@@ -17,11 +17,15 @@
 
 #include "kudu/security/token_signing_key.h"
 
+#include <memory>
+
 #include <glog/logging.h>
 
 #include "kudu/security/token.pb.h"
 #include "kudu/util/status.h"
 
+using std::unique_ptr;
+
 namespace kudu {
 namespace security {
 
@@ -39,7 +43,7 @@ bool TokenSigningPublicKey::VerifySignature(const SignedTokenPB& token) const {
 }
 
 TokenSigningPrivateKey::TokenSigningPrivateKey(
-    int64_t key_seq_num, int64_t expire_time, Key key)
+    int64_t key_seq_num, int64_t expire_time, unique_ptr<PrivateKey> key)
     : key_(std::move(key)),
       key_seq_num_(key_seq_num),
       expire_time_(expire_time) {

http://git-wip-us.apache.org/repos/asf/kudu/blob/d91313d6/src/kudu/security/token_signing_key.h
----------------------------------------------------------------------
diff --git a/src/kudu/security/token_signing_key.h b/src/kudu/security/token_signing_key.h
index 38fa795..540c847 100644
--- a/src/kudu/security/token_signing_key.h
+++ b/src/kudu/security/token_signing_key.h
@@ -16,7 +16,10 @@
 // under the License.
 #pragma once
 
+#include <memory>
+
 #include "kudu/gutil/macros.h"
+#include "kudu/security/crypto.h"
 #include "kudu/security/openssl_util.h"
 #include "kudu/security/token.pb.h"
 #include "kudu/util/status.h"
@@ -56,7 +59,9 @@ class TokenSigningPublicKey {
 // number and expiration date.
 class TokenSigningPrivateKey {
  public:
-  TokenSigningPrivateKey(int64_t key_seq_num, int64_t expire_time, Key key);
+  TokenSigningPrivateKey(int64_t key_seq_num,
+                         int64_t expire_time,
+                         std::unique_ptr<PrivateKey> key);
   ~TokenSigningPrivateKey();
 
   // Sign a token, and store the signature and signing key's sequence number.
@@ -65,7 +70,7 @@ class TokenSigningPrivateKey {
   // Export the public-key portion of this signing key.
   void ExportPublicKeyPB(TokenSigningPublicKeyPB* pb);
  private:
-  Key key_;
+  std::unique_ptr<PrivateKey> key_;
   int64_t key_seq_num_;
   int64_t expire_time_;