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

[1/2] kudu git commit: docs: clarify guidelines on boost usage

Repository: kudu
Updated Branches:
  refs/heads/master c8ca20fa7 -> 4a5f1392b


docs: clarify guidelines on boost usage

Change-Id: I788ac1426a6a79192e1cdd88892cb2fa1a978b47
Reviewed-on: http://gerrit.cloudera.org:8080/5752
Reviewed-by: David Ribeiro Alves <dr...@apache.org>
Tested-by: Kudu Jenkins
Reviewed-by: Dan Burkert <da...@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/36445904
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/36445904
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/36445904

Branch: refs/heads/master
Commit: 36445904cb16b69a01b0ae7ddb55144197ad44a7
Parents: c8ca20f
Author: Todd Lipcon <to...@apache.org>
Authored: Fri Jan 20 11:54:15 2017 -0800
Committer: Todd Lipcon <to...@apache.org>
Committed: Sun Jan 22 23:57:59 2017 +0000

----------------------------------------------------------------------
 docs/contributing.adoc | 13 ++++++++++---
 1 file changed, 10 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/36445904/docs/contributing.adoc
----------------------------------------------------------------------
diff --git a/docs/contributing.adoc b/docs/contributing.adoc
index 56b9cb3..f666b2c 100644
--- a/docs/contributing.adoc
+++ b/docs/contributing.adoc
@@ -162,13 +162,20 @@ away from `foo.Pass()` in favor of `std::move(foo)`.
 `boost` classes from header-only libraries can be used in cases where a suitable
 replacement does not exist in the Kudu code base. However:
 
-* Do not introduce dependencies on `boost` classes not already depended on. We
-  are actively trying to remove `boost` from the Kudu code base, and new
-  dependencies make that work more difficult.
+* Do not introduce dependencies on `boost` classes where equivalent functionality
+  exists in the standard {cpp} library or in `src/kudu/gutil/`. For example, prefer
+  `strings::Split()` from `gutil` rather than `boost::split`.
+* Prefer using functionality from  `boost` rather than re-implementing the same
+  functionality, _unless_ using the `boost` functionality requires excessive use of
+  {cpp} features which are disallowed by our style guidelines. For example,
+  `boost::spirit` is heavily based on template metaprogramming and should not be used.
 * Do not use `boost` in any public headers for the Kudu {cpp} client, because
   `boost` commonly breaks backward compatibility, and passing data between two
   `boost` versions (one by the user, one by Kudu) causes serious issues.
 
+When in doubt about introducing a new dependency on any `boost` functionality,
+it is best to email `dev@kudu.apache.org` to start a discussion.
+
 === Line length
 
 The Kudu team allows line lengths of 100 characters per line, rather than Google's standard of 80. Try to


[2/2] kudu git commit: [security] groundwork for cert signing service

Posted by to...@apache.org.
[security] groundwork for cert signing service

Added code for TLS certificate management in the scope of generating
root CA and Tablet Server certificates for Kudu.  The code provides
wrappers for OpenSSL functions with Kudu-specific interface.

The code is compiled into a library which Kudu's X509 CSR server
and client should be linked with (master is going to be a CSR server
and tablet server is going to be a CSR client).

Unit tests are adedd as well.

Change-Id: Ic2ce55d38f4d06172fadaaa702f4550997d9bc8f
Reviewed-on: http://gerrit.cloudera.org:8080/5671
Tested-by: Kudu Jenkins
Reviewed-by: Todd Lipcon <to...@apache.org>


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

Branch: refs/heads/master
Commit: 4a5f1392bbddb6a5eefd5fa23c098e7dc1a027d0
Parents: 3644590
Author: Alexey Serbin <as...@cloudera.com>
Authored: Tue Jan 10 17:03:20 2017 -0800
Committer: Todd Lipcon <to...@apache.org>
Committed: Mon Jan 23 04:16:36 2017 +0000

----------------------------------------------------------------------
 src/kudu/security/CMakeLists.txt               |   9 +-
 src/kudu/security/ca/cert_management.cc        | 649 ++++++++++++++++++++
 src/kudu/security/ca/cert_management.h         | 256 ++++++++
 src/kudu/security/openssl_util.cc              |   2 +-
 src/kudu/security/test/cert_management-test.cc | 492 +++++++++++++++
 src/kudu/security/test/test_certs.cc           | 212 +++++++
 src/kudu/security/test/test_certs.h            |  45 ++
 7 files changed, 1661 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/4a5f1392/src/kudu/security/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/kudu/security/CMakeLists.txt b/src/kudu/security/CMakeLists.txt
index 68a71e2..d20b73b 100644
--- a/src/kudu/security/CMakeLists.txt
+++ b/src/kudu/security/CMakeLists.txt
@@ -30,6 +30,7 @@ if (${OPENSSL_VERSION} VERSION_LESS "1.0.2")
 endif()
 
 set(SECURITY_SRCS
+  ca/cert_management.cc
   init.cc
   openssl_util.cc
   ${PORTED_X509_CHECK_HOST_CC}
@@ -49,7 +50,8 @@ ADD_EXPORTABLE_LIBRARY(security
 
 if (NOT NO_TESTS)
   set(SECURITY_TEST_SRCS
-    test/mini_kdc.cc)
+    test/mini_kdc.cc
+    test/test_certs.cc)
 
   add_library(security-test ${SECURITY_TEST_SRCS})
   target_link_libraries(security-test
@@ -62,6 +64,7 @@ if (NOT NO_TESTS)
     security
     security-test
     ${KUDU_MIN_TEST_LIBS})
-endif()
 
-ADD_KUDU_TEST(test/mini_kdc-test)
+  ADD_KUDU_TEST(test/mini_kdc-test)
+  ADD_KUDU_TEST(test/cert_management-test)
+endif()

http://git-wip-us.apache.org/repos/asf/kudu/blob/4a5f1392/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
new file mode 100644
index 0000000..c9c3b57
--- /dev/null
+++ b/src/kudu/security/ca/cert_management.cc
@@ -0,0 +1,649 @@
+// 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 <cstdio>
+#include <cstdlib>
+#include <functional>
+#include <iostream>
+#include <memory>
+#include <sstream>
+#include <string>
+#include <type_traits>
+
+#include <glog/logging.h>
+#include <openssl/conf.h>
+#ifndef OPENSSL_NO_ENGINE
+#include <openssl/engine.h>
+#endif
+#include <openssl/pem.h>
+#include <openssl/x509.h>
+#include <openssl/x509v3.h>
+
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/security/openssl_util.h"
+#include "kudu/util/scoped_cleanup.h"
+#include "kudu/util/status.h"
+
+using std::lock_guard;
+using std::move;
+using std::ostringstream;
+using std::string;
+using strings::Substitute;
+
+#define CERT_CHECK_OK(call) \
+  CHECK_GT((call), 0)
+
+#define CERT_RET_NOT_OK(call, msg) \
+  if ((call) <= 0) { \
+    return Status::RuntimeError(Substitute("$0: $1", \
+        (msg), GetOpenSSLErrors())); \
+  }
+
+#define CERT_RET_IF_NULL(call, msg) \
+  if ((call) == nullptr) { \
+    return Status::RuntimeError(Substitute("$0: $1", \
+        (msg), GetOpenSSLErrors())); \
+  }
+
+namespace kudu {
+namespace security {
+namespace ca {
+
+const string& DataFormatToString(DataFormat fmt) {
+  static const string kStrFormatUnknown = "UNKNOWN";
+  static const string kStrFormatDer = "DER";
+  static const string kStrFormatPem = "PEM";
+  switch (fmt) {
+    case DataFormat::DER:
+      return kStrFormatDer;
+    case DataFormat::PEM:
+      return kStrFormatPem;
+    default:
+      return kStrFormatUnknown;
+  }
+}
+
+Status BasicWrapper::FromFile(const string& fpath, DataFormat format) {
+  c_unique_ptr<BIO> bio(BIO_new(BIO_s_file()), BIO_free);
+  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());
+  c_unique_ptr<BIO> bio(BIO_new_mem_buf(
+#if OPENSSL_VERSION_NUMBER < 0x10002000L
+      const_cast<void*>(mdata),
+#else
+      mdata,
+#endif
+      data.size()), BIO_free);
+  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);
+  c_unique_ptr<BIO> bio(BIO_new(BIO_s_mem()), BIO_free);
+  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) {
+  c_unique_ptr<RawDataType> d(data, EVP_PKEY_free);
+  data_.swap(d);
+}
+
+Status Key::FromBIO(BIO* bio, DataFormat format) {
+  CHECK(bio);
+  c_unique_ptr<EVP_PKEY> key(nullptr, EVP_PKEY_free);
+  switch (format) {
+    case DataFormat::DER:
+      key.reset(d2i_PrivateKey_bio(bio, nullptr));
+      break;
+    case DataFormat::PEM:
+      key.reset(PEM_read_bio_PrivateKey(bio, nullptr, nullptr, nullptr));
+      break;
+    default:
+      return Status::InvalidArgument(Substitute("$0: unrecognized format",
+                                                static_cast<uint16_t>(format)));
+  }
+  if (PREDICT_FALSE(!key)) {
+    return Status::RuntimeError(
+        Substitute("error reading private key: $0", GetOpenSSLErrors()));
+  }
+  key.swap(data_);
+  return Status::OK();
+}
+
+Status Key::ToBIO(BIO* bio, DataFormat format) const {
+  CHECK(bio);
+  CHECK(data_);
+  switch (format) {
+    case DataFormat::DER:
+      CERT_RET_NOT_OK(i2d_PrivateKey_bio(bio, data_.get()),
+                      "error exporting private key in DER format");
+      break;
+    case DataFormat::PEM:
+      {
+        c_unique_ptr<RSA> rsa(EVP_PKEY_get1_RSA(data_.get()), RSA_free);
+        CERT_RET_NOT_OK(PEM_write_bio_RSAPrivateKey(bio, rsa.get(),
+                                                    nullptr, nullptr, 0,
+                                                    nullptr, nullptr),
+                        "error exporting private key in PEM format");
+      }
+      break;
+    default:
+      return Status::InvalidArgument(Substitute("$0: unrecognized format",
+                                                static_cast<uint16_t>(format)));
+  }
+  CERT_RET_NOT_OK(BIO_flush(bio), "error flushing BIO");
+  return Status::OK();
+}
+
+void Cert::AdoptRawData(RawDataType* data) {
+  c_unique_ptr<RawDataType> d(data, X509_free);
+  data_.swap(d);
+}
+
+Status Cert::FromBIO(BIO* bio, DataFormat format) {
+  CHECK(bio);
+  c_unique_ptr<X509> cert(nullptr, X509_free);
+  switch (format) {
+    case DataFormat::DER:
+      cert.reset(d2i_X509_bio(bio, nullptr));
+      break;
+    case DataFormat::PEM:
+      cert.reset(PEM_read_bio_X509(bio, nullptr, nullptr, nullptr));
+      break;
+    default: {
+      string err_msg = Substitute("$0: unrecognized data format",
+                                  static_cast<uint16_t>(format));
+      LOG(DFATAL) << err_msg;
+      return Status::InvalidArgument(err_msg);
+    }
+  }
+  if (PREDICT_FALSE(!cert)) {
+    return Status::RuntimeError(
+        Substitute("error loading X509 certificate: $0", GetOpenSSLErrors()));
+  }
+  cert.swap(data_);
+  return Status::OK();
+}
+
+Status Cert::ToBIO(BIO* bio, DataFormat format) const {
+  CHECK(data_);
+  CHECK(bio);
+  switch (format) {
+    case DataFormat::DER:
+      CERT_RET_NOT_OK(i2d_X509_bio(bio, data_.get()),
+                      "error exporting X509 certificate in DER format");
+      break;
+    case DataFormat::PEM:
+      CERT_RET_NOT_OK(PEM_write_bio_X509(bio, data_.get()),
+                      "error exporting X509 certificate in PEM format");
+      break;
+    default:
+      return Status::InvalidArgument(Substitute("$0: unrecognized format",
+                                                static_cast<uint16_t>(format)));
+  }
+  CERT_RET_NOT_OK(BIO_flush(bio), "error flushing BIO");
+  return Status::OK();
+}
+
+void CertSignRequest::AdoptRawData(RawDataType* data) {
+  c_unique_ptr<RawDataType> d(data, X509_REQ_free);
+  data_.swap(d);
+}
+
+Status CertSignRequest::FromBIO(BIO* bio, DataFormat format) {
+  CHECK(bio);
+  c_unique_ptr<X509_REQ> req(nullptr, X509_REQ_free);
+  switch (format) {
+    case DataFormat::DER:
+      req.reset(d2i_X509_REQ_bio(bio, nullptr));
+      break;
+    case DataFormat::PEM:
+      req.reset(PEM_read_bio_X509_REQ(bio, nullptr, nullptr, nullptr));
+      break;
+    default: {
+      string err_msg = Substitute("$0: unrecognized data format",
+                                  static_cast<uint16_t>(format));
+      LOG(DFATAL) << err_msg;
+      return Status::InvalidArgument(err_msg);
+    }
+  }
+  if (PREDICT_FALSE(!req)) {
+    return Status::RuntimeError(
+        Substitute("error loading X509 CSR: $0", GetOpenSSLErrors()));
+  }
+  req.swap(data_);
+  return Status::OK();
+}
+
+Status CertSignRequest::ToBIO(BIO* bio, DataFormat format) const {
+  CHECK(bio);
+  CHECK(data_);
+  switch (format) {
+    case DataFormat::DER:
+      CERT_RET_NOT_OK(i2d_X509_REQ_bio(bio, data_.get()),
+                      "error exporting X509 CSR in DER format");
+      break;
+    case DataFormat::PEM:
+      CERT_RET_NOT_OK(PEM_write_bio_X509_REQ(bio, data_.get()),
+                      "error exporting X509 CSR in PEM format");
+      break;
+    default:
+      return Status::InvalidArgument(Substitute("$0: unrecognized format",
+                                                static_cast<uint16_t>(format)));
+  }
+  CERT_RET_NOT_OK(BIO_flush(bio), "error flushing BIO");
+  return Status::OK();
+}
+
+Status GeneratePrivateKey(int num_bits, Key* ret) {
+  CHECK(ret);
+  InitializeOpenSSL();
+  c_unique_ptr<EVP_PKEY> key(EVP_PKEY_new(), EVP_PKEY_free);
+  {
+    c_unique_ptr<BIGNUM> bn(BN_new(), BN_free);
+    CERT_CHECK_OK(BN_set_word(bn.get(), RSA_F4));
+    c_unique_ptr<RSA> rsa(RSA_new(), RSA_free);
+    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();
+}
+
+CertRequestGeneratorBase::CertRequestGeneratorBase(Config config)
+    : config_(move(config)) {
+}
+
+CertRequestGenerator::~CertRequestGenerator() {
+  sk_X509_EXTENSION_pop_free(extensions_, X509_EXTENSION_free);
+}
+
+Status CertRequestGeneratorBase::GenerateRequest(const Key& key,
+                                                 CertSignRequest* ret) const {
+  CHECK(ret);
+  CHECK(Initialized());
+  c_unique_ptr<X509_REQ> req(X509_REQ_new(), X509_REQ_free);
+  CERT_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);
+
+#define CERT_SET_SUBJ_FIELD(field, code, err_msg) \
+  do { \
+    const string& f = (field); \
+    if (!f.empty()) { \
+      CERT_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)); \
+    } \
+  } while (false)
+
+  CERT_SET_SUBJ_FIELD(config_.country, "C", "country");
+  CERT_SET_SUBJ_FIELD(config_.state, "ST", "state");
+  CERT_SET_SUBJ_FIELD(config_.locality, "L", "locality/city");
+  CERT_SET_SUBJ_FIELD(config_.org, "O", "organization");
+  CERT_SET_SUBJ_FIELD(config_.unit, "OU", "organizational unit");
+  CERT_SET_SUBJ_FIELD(config_.uuid, "CN", "common name");
+#undef CERT_SET_SUBJ_FIELD
+
+  // Set necessary extensions into the request.
+  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");
+  ret->AdoptRawData(req.release());
+
+  return Status::OK();
+}
+
+Status CertRequestGeneratorBase::PushExtension(stack_st_X509_EXTENSION* st,
+                                               int32_t nid, const char* value) {
+  c_unique_ptr<X509_EXTENSION> ex(
+      X509V3_EXT_conf_nid(nullptr, nullptr, nid, const_cast<char*>(value)),
+      X509_EXTENSION_free);
+  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");
+  return Status::OK();
+}
+
+CertRequestGenerator::CertRequestGenerator(Config config)
+    : CertRequestGeneratorBase(config),
+      extensions_(nullptr),
+      is_initialized_(false) {
+}
+
+Status CertRequestGenerator::Init() {
+  InitializeOpenSSL();
+
+  lock_guard<simple_spinlock> guard(lock_);
+  if (is_initialized_) {
+    return Status::OK();
+  }
+  if (config_.uuid.empty()) {
+    return Status::InvalidArgument("missing end-entity UUID/name");
+  }
+  // Check that the config contain at least one entity (DNS name/IP address)
+  // to bind the generated certificate.
+  if (config_.hostnames.empty() && config_.ips.empty()) {
+    return Status::InvalidArgument("SAN: missing DNS names and IP addresses");
+  }
+
+  extensions_ = sk_X509_EXTENSION_new_null();
+
+  // Permitted usages for the generated keys is set via X509 V3
+  // standard/extended key usage attributes.
+  // See https://www.openssl.org/docs/man1.0.1/apps/x509v3_config.html
+  // for details.
+
+  // The generated certificates are for using as TLS certificates for
+  // both client and server.
+  RETURN_NOT_OK(PushExtension(extensions_, NID_key_usage,
+                              "critical,digitalSignature,keyEncipherment"));
+  // The generated certificates should be good for authentication
+  // of a server to a client and vice versa: the intended users of the
+  // certificates are tablet servers which are going to talk to master
+  // and other tablet servers via TLS channels.
+  RETURN_NOT_OK(PushExtension(extensions_, NID_ext_key_usage,
+                              "critical,serverAuth,clientAuth"));
+  // The generated certificates are not intended to be used as CA certificates
+  // (i.e. they cannot be used to sign/issue certificates).
+  RETURN_NOT_OK(PushExtension(extensions_, NID_basic_constraints,
+                              "critical,CA:FALSE"));
+  ostringstream san_hosts;
+  for (size_t i = 0; i < config_.hostnames.size(); ++i) {
+    const string& hostname = config_.hostnames[i];
+    if (hostname.empty()) {
+      // Basic validation: check for emptyness. Probably, more advanced
+      // validation is needed here.
+      return Status::InvalidArgument("SAN: an empty hostname");
+    }
+    if (i != 0) {
+      san_hosts << ",";
+    }
+    san_hosts << "DNS." << i << ":" << hostname;
+  }
+  ostringstream san_ips;
+  for (size_t i = 0; i < config_.ips.size(); ++i) {
+    const string& ip = config_.ips[i];
+    if (ip.empty()) {
+      // Basic validation: check for emptyness. Probably, more advanced
+      // validation is needed here.
+      return Status::InvalidArgument("SAN: an empty IP address");
+    }
+    if (i != 0) {
+      san_ips << ",";
+    }
+    san_ips << "IP." << i << ":" << ip;
+  }
+  // Encode hostname and IP address into the subjectAlternativeName attribute.
+  const string alt_name = san_hosts.str() +
+      ((!san_hosts.str().empty() && !san_ips.str().empty()) ? "," : "") +
+      san_ips.str();
+  RETURN_NOT_OK(PushExtension(extensions_, NID_subject_alt_name,
+                              alt_name.c_str()));
+  if (!config_.comment.empty()) {
+    // Add the comment if it's not empty.
+    RETURN_NOT_OK(PushExtension(extensions_, NID_netscape_comment,
+                                config_.comment.c_str()));
+  }
+  is_initialized_ = true;
+
+  return Status::OK();
+}
+
+bool CertRequestGenerator::Initialized() const {
+  lock_guard<simple_spinlock> guard(lock_);
+  return is_initialized_;
+}
+
+Status CertRequestGenerator::SetExtensions(X509_REQ* req) const {
+  CERT_RET_NOT_OK(X509_REQ_add_extensions(req, extensions_),
+                  "error setting X509 request extensions");
+  return Status::OK();
+}
+
+CaCertRequestGenerator::CaCertRequestGenerator(Config config)
+    : CertRequestGeneratorBase(config),
+      extensions_(nullptr),
+      is_initialized_(false) {
+}
+
+CaCertRequestGenerator::~CaCertRequestGenerator() {
+  sk_X509_EXTENSION_pop_free(extensions_, X509_EXTENSION_free);
+}
+
+Status CaCertRequestGenerator::Init() {
+  InitializeOpenSSL();
+
+  lock_guard<simple_spinlock> guard(lock_);
+  if (is_initialized_) {
+    return Status::OK();
+  }
+  if (config_.uuid.empty()) {
+    return Status::InvalidArgument("missing CA service UUID/name");
+  }
+
+  extensions_ = sk_X509_EXTENSION_new_null();
+
+  // Permitted usages for the generated keys is set via X509 V3
+  // standard/extended key usage attributes.
+  // See https://www.openssl.org/docs/man1.0.1/apps/x509v3_config.html
+  // for details.
+
+  // The target ceritifcate is a CA certificate: it's for signing X509 certs.
+  RETURN_NOT_OK(PushExtension(extensions_, NID_key_usage,
+                              "critical,keyCertSign"));
+  // The generated certificates are for the private CA service.
+  RETURN_NOT_OK(PushExtension(extensions_, NID_basic_constraints,
+                              "critical,CA:TRUE"));
+  if (!config_.comment.empty()) {
+    // Add the comment if it's not empty.
+    RETURN_NOT_OK(PushExtension(extensions_, NID_netscape_comment,
+                                config_.comment.c_str()));
+  }
+  is_initialized_ = true;
+
+  return Status::OK();
+}
+
+bool CaCertRequestGenerator::Initialized() const {
+  lock_guard<simple_spinlock> guard(lock_);
+  return is_initialized_;
+}
+
+Status CaCertRequestGenerator::SetExtensions(X509_REQ* req) const {
+  CERT_RET_NOT_OK(X509_REQ_add_extensions(req, extensions_),
+                  "error setting X509 request extensions");
+  return Status::OK();
+}
+
+CertSigner::CertSigner(Config config)
+    : config_(move(config)),
+      is_initialized_(false) {
+}
+
+Status CertSigner::Init() {
+  InitializeOpenSSL();
+
+  lock_guard<simple_spinlock> guard(lock_);
+  if (is_initialized_) {
+    return Status::OK();
+  }
+  RETURN_NOT_OK(ca_cert_.FromFile(config_.ca_cert_path, DataFormat::PEM));
+  RETURN_NOT_OK(ca_private_key_.FromFile(config_.ca_private_key_path,
+                DataFormat::PEM));
+  CERT_RET_NOT_OK(X509_check_private_key(ca_cert_.GetRawData(),
+                                         ca_private_key_.GetRawData()),
+                  Substitute("$0, $1: CA certificate and private key "
+                             "do not match",
+                             config_.ca_cert_path, config_.ca_private_key_path));
+  is_initialized_ = true;
+  return Status::OK();
+}
+
+bool CertSigner::Initialized() const {
+  lock_guard<simple_spinlock> guard(lock_);
+  return is_initialized_;
+}
+
+const Cert& CertSigner::ca_cert() const {
+  lock_guard<simple_spinlock> guard(lock_);
+  DCHECK(is_initialized_);
+  return ca_cert_;
+}
+
+const Key& CertSigner::ca_private_key() const {
+  lock_guard<simple_spinlock> guard(lock_);
+  DCHECK(is_initialized_);
+  return ca_private_key_;
+}
+
+Status CertSigner::Sign(const CertSignRequest& req, Cert* ret) const {
+  CHECK(ret);
+  CHECK(Initialized());
+  c_unique_ptr<X509> x509(X509_new(), X509_free);
+  RETURN_NOT_OK(FillCertTemplateFromRequest(req.GetRawData(), x509.get()));
+  RETURN_NOT_OK(DoSign(EVP_sha256(), config_.exp_interval_sec, x509.get()));
+  ret->AdoptRawData(x509.release());
+
+  return Status::OK();
+}
+
+// This is modeled after code in copy_extensions() function from
+// $OPENSSL_ROOT/apps/apps.c with OpenSSL 1.0.2.
+Status CertSigner::CopyExtensions(X509_REQ* req, X509* x) {
+  CHECK(req);
+  CHECK(x);
+  STACK_OF(X509_EXTENSION)* exts = X509_REQ_get_extensions(req);
+  auto exts_cleanup = MakeScopedCleanup([&exts]() {
+    sk_X509_EXTENSION_pop_free(exts, X509_EXTENSION_free);
+  });
+  for (size_t i = 0; i < sk_X509_EXTENSION_num(exts); ++i) {
+    X509_EXTENSION* ext = sk_X509_EXTENSION_value(exts, i);
+    ASN1_OBJECT* obj = X509_EXTENSION_get_object(ext);
+    int32_t idx = X509_get_ext_by_OBJ(x, obj, -1);
+    if (idx != -1) {
+      // If extension exits, delete all extensions of same type.
+      do {
+        c_unique_ptr<X509_EXTENSION> tmpext(X509_get_ext(x, idx),
+                                       X509_EXTENSION_free);
+        X509_delete_ext(x, idx);
+        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");
+  }
+
+  return Status::OK();
+}
+
+Status CertSigner::FillCertTemplateFromRequest(X509_REQ* req, X509* tmpl) {
+  CHECK(req);
+  if (!req->req_info ||
+      !req->req_info->pubkey ||
+      !req->req_info->pubkey->public_key ||
+      !req->req_info->pubkey->public_key->data) {
+    return Status::RuntimeError("corrupted CSR: no public key");
+  }
+  c_unique_ptr<EVP_PKEY> pub_key(X509_REQ_get_pubkey(req), EVP_PKEY_free);
+  if (!pub_key) {
+    return Status::RuntimeError("error unpacking public key from CSR");
+  }
+  const int rc = X509_REQ_verify(req, pub_key.get());
+  if (rc < 0) {
+    return Status::RuntimeError("CSR signature verification error");
+  }
+  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");
+  RETURN_NOT_OK(CopyExtensions(req, tmpl));
+  CERT_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");
+  return Status::OK();
+}
+
+Status CertSigner::GenerateSerial(c_unique_ptr<ASN1_INTEGER>* ret) {
+  c_unique_ptr<BIGNUM> btmp(BN_new(), BN_free);
+  CERT_RET_NOT_OK(BN_pseudo_rand(btmp.get(), 64, 0, 0),
+                  "error generating random number");
+  c_unique_ptr<ASN1_INTEGER> serial(ASN1_INTEGER_new(), ASN1_INTEGER_free);
+  CERT_RET_IF_NULL(BN_to_ASN1_INTEGER(btmp.get(), serial.get()),
+                   "error converting number into ASN1 representation");
+  if (ret) {
+    ret->swap(serial);
+  }
+  return Status::OK();
+}
+
+Status CertSigner::DoSign(const EVP_MD* digest, int32_t exp_seconds,
+                          X509* ret) const {
+  // Version 3 (v3) of X509 certificates. The integer value is one less
+  // than the version it represents. This is not a typo. :)
+  static const int kX509V3 = 2;
+
+  CERT_RET_NOT_OK(
+      X509_set_issuer_name(CHECK_NOTNULL(ret),
+                           X509_get_subject_name(ca_cert_.GetRawData())),
+      "error setting issuer name");
+  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");
+  RETURN_NOT_OK(DigestSign(digest, ca_private_key_.GetRawData(), ret));
+
+  return Status::OK();
+}
+
+} // namespace ca
+} // namespace security
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/4a5f1392/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
new file mode 100644
index 0000000..65faafd
--- /dev/null
+++ b/src/kudu/security/ca/cert_management.h
@@ -0,0 +1,256 @@
+// 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 <functional>
+#include <map>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "kudu/gutil/macros.h"
+#include "kudu/security/openssl_util.h"
+#include "kudu/util/locks.h"
+#include "kudu/util/status.h"
+
+// Forward declarations for the relevant OpenSSL typedefs
+// in addition to openssl_util.h.
+typedef struct asn1_string_st ASN1_INTEGER;
+typedef struct env_md_st EVP_MD;
+typedef struct evp_pkey_st EVP_PKEY;
+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 {
+namespace ca {
+
+template <typename T>
+using c_unique_ptr = std::unique_ptr<T, std::function<void(T*)>>;
+
+// Acceptable formats for X509 certificates, X509 CSRs, and private keys.
+enum class DataFormat {
+  DER = 0,    // DER/ASN1 format (binary)
+  PEM = 1,    // PEM format (ASCII)
+};
+
+// 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 {
+ public:
+  typedef X509 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* 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);
+
+// Base utility class for issuing X509 CSRs.
+class CertRequestGeneratorBase {
+ public:
+  // Properties for the generated X509 CSR.  Using server UUID for the common
+  // name field.
+  struct Config {
+    const std::string country;  // subject field: C
+    const std::string state;    // subject field: ST
+    const std::string locality; // subject field: L
+    const std::string org;      // subject field: O
+    const std::string unit;     // subject field: OU
+    const std::string uuid;     // subject field: CN
+    const std::string comment;  // custom extension: Netscape Comment
+    const std::vector<std::string> hostnames; // subjectAltName extension (DNS:)
+    const std::vector<std::string> ips;       // subjectAltName extension (IP:)
+  };
+
+  explicit CertRequestGeneratorBase(Config config);
+  virtual ~CertRequestGeneratorBase() = default;
+
+  virtual Status Init() = 0;
+  virtual bool Initialized() const = 0;
+
+  // Generate X509 CSR using the specified key. To obtain the key,
+  // call the GeneratePrivateKey() function.
+  Status GenerateRequest(const Key& key, CertSignRequest* ret) const;
+
+ protected:
+  // Push the specified extension into the stack provided.
+  static Status PushExtension(stack_st_X509_EXTENSION* st, int32_t nid,
+                              const char* value);
+  // Set the certificate-specific extensions into the specified request.
+  virtual Status SetExtensions(X509_REQ* req) const = 0;
+
+  const Config config_;
+
+ private:
+  DISALLOW_COPY_AND_ASSIGN(CertRequestGeneratorBase);
+};
+
+// An utility class that facilitates issuing certificate signing requests
+// (a.k.a. X509 CSRs).
+class CertRequestGenerator : public CertRequestGeneratorBase {
+ public:
+  // The CertRequestGenerator object is bound to the server UUID, hostnames
+  // and IP addresses specified by the 'config' parameter. The hostnames and
+  // IP addresses are put into the X509v3 SAN extension (subject alternative
+  // name, a.k.a. subjectAltName). The SAN can be used while verifying the
+  // generated certificates during TLS handshake.
+  explicit CertRequestGenerator(Config config);
+  ~CertRequestGenerator();
+
+  Status Init() override;
+  bool Initialized() const override;
+
+ protected:
+  Status SetExtensions(X509_REQ* req) const override;
+
+ private:
+  stack_st_X509_EXTENSION* extensions_;
+  mutable simple_spinlock lock_;
+  bool is_initialized_; // protected by lock_
+};
+
+// An utility class that facilitates issuing of root CA self-signed certificate
+// signing requests.
+class CaCertRequestGenerator : public CertRequestGeneratorBase {
+ public:
+  explicit CaCertRequestGenerator(Config config);
+  ~CaCertRequestGenerator();
+
+  Status Init() override;
+  bool Initialized() const override;
+
+ protected:
+  Status SetExtensions(X509_REQ* req) const override;
+
+ private:
+  stack_st_X509_EXTENSION* extensions_;
+  mutable simple_spinlock lock_;
+  bool is_initialized_; // protected by lock_
+};
+
+// An utility class for issuing and signing certificates.
+class CertSigner {
+ public:
+  struct Config {
+    const int32_t exp_interval_sec;
+    const std::string ca_cert_path;
+    const std::string ca_private_key_path;
+  };
+
+  explicit CertSigner(Config config);
+  ~CertSigner() = default;
+
+  Status Init();
+  bool Initialized() const;
+
+  const Cert& ca_cert() const;
+  const Key& ca_private_key() const;
+
+  Status Sign(const CertSignRequest& req, Cert* ret) const;
+
+ private:
+  static Status CopyExtensions(X509_REQ* req, X509* x);
+  static Status FillCertTemplateFromRequest(X509_REQ* req, X509* tmpl);
+  static Status DigestSign(const EVP_MD* md, EVP_PKEY* pkey, X509* x);
+  static Status GenerateSerial(c_unique_ptr<ASN1_INTEGER>* ret);
+
+  Status DoSign(const EVP_MD* digest, int32_t exp_seconds, X509 *ret) const;
+
+  const std::string ca_cert_path_;
+  const std::string ca_private_key_path_;
+  const Config config_;
+  mutable simple_spinlock lock_;
+  bool is_initialized_; // protected by lock_
+  Cert ca_cert_;
+  Key ca_private_key_;
+
+  DISALLOW_COPY_AND_ASSIGN(CertSigner);
+};
+
+} // namespace ca
+} // namespace security
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/4a5f1392/src/kudu/security/openssl_util.cc
----------------------------------------------------------------------
diff --git a/src/kudu/security/openssl_util.cc b/src/kudu/security/openssl_util.cc
index 11d9b5c..c6699db 100644
--- a/src/kudu/security/openssl_util.cc
+++ b/src/kudu/security/openssl_util.cc
@@ -59,8 +59,8 @@ void ThreadIdCB(CRYPTO_THREADID* tid) {
 }
 
 void DoInitializeOpenSSL() {
-  SSL_library_init();
   SSL_load_error_strings();
+  SSL_library_init();
   OpenSSL_add_all_algorithms();
   RAND_poll();
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/4a5f1392/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
new file mode 100644
index 0000000..0273248
--- /dev/null
+++ b/src/kudu/security/test/cert_management-test.cc
@@ -0,0 +1,492 @@
+// 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());
+    }
+    const CertSigner::Config signer_config = {
+      .exp_interval_sec = 24 * 60 * 60,
+      .ca_cert_path = ca_cert_file_,
+      .ca_private_key_path = ca_private_key_file_,
+    };
+    CertSigner signer_shared(signer_config);
+    if (SINGLE_INIT == init_type) {
+      ASSERT_OK(signer_shared.Init());
+    }
+
+    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(signer_config);
+
+          CertRequestGenerator& gen = (SHARED == sharing_type) ? gen_shared
+                                                               : gen_local;
+          CertSigner& signer = (SHARED == sharing_type) ? signer_shared
+                                                        : signer_local;
+
+          if (DEDICATED == sharing_type || MULTIPLE_INIT == init_type) {
+            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 || MULTIPLE_INIT == init_type) {
+            CHECK_OK(signer.Init());
+          }
+          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.
+  const CertSigner::Config config = {
+    .exp_interval_sec = 24 * 60 * 60,
+    .ca_cert_path = "/bin/sh",
+    .ca_private_key_path = "/bin/cat",
+  };
+  CertSigner signer(config);
+  ASSERT_FALSE(signer.Init().ok());
+}
+
+// Check that CertSigner behaves in a predictable way if given non-matching
+// CA private key and certificate.
+TEST_F(CertManagementTest, SignerInitWithMismatchedCertAndKey) {
+  {
+    const CertSigner::Config config = {
+      .exp_interval_sec = 24 * 60 * 60,
+      .ca_cert_path = ca_cert_file_,
+      .ca_private_key_path = ca_exp_private_key_file_,
+    };
+    CertSigner signer(config);
+    Status s = signer.Init();
+    const string err_msg = s.ToString();
+    ASSERT_TRUE(s.IsRuntimeError()) << err_msg;
+    ASSERT_STR_CONTAINS(err_msg, "CA certificate and private key do not match");
+  }
+  {
+    const CertSigner::Config config = {
+      .exp_interval_sec = 24 * 60 * 60,
+      .ca_cert_path = ca_exp_cert_file_,
+      .ca_private_key_path = ca_private_key_file_,
+    };
+    CertSigner signer(config);
+    Status s = signer.Init();
+    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));
+  const CertSigner::Config signer_config = {
+    .exp_interval_sec = 24 * 60 * 60,
+    .ca_cert_path = ca_exp_cert_file_,
+    .ca_private_key_path = ca_exp_private_key_file_,
+  };
+  CertSigner signer(signer_config);
+  // Even if the certificate is expired, the signer should initialize OK.
+  ASSERT_OK(signer.Init());
+  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));
+  const CertSigner::Config signer_config = {
+    .exp_interval_sec = 24 * 60 * 60,
+    .ca_cert_path = ca_cert_file_,
+    .ca_private_key_path = ca_private_key_file_,
+  };
+  CertSigner signer(signer_config);
+  ASSERT_OK(signer.Init());
+  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));
+  const CertSigner::Config signer_config = {
+    .exp_interval_sec = 24 * 60 * 60,
+    .ca_cert_path = ca_cert_file_,
+    .ca_private_key_path = ca_private_key_file_,
+  };
+  CertSigner signer(signer_config);
+  ASSERT_OK(signer.Init());
+  Cert cert;
+  ASSERT_OK(signer.Sign(req, &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));
+
+  const CertSigner::Config signer_config = {
+    .exp_interval_sec = 24 * 60 * 60,
+    .ca_cert_path = ca_cert_file_,
+    .ca_private_key_path = ca_private_key_file_,
+  };
+  CertSigner signer(signer_config);
+  ASSERT_OK(signer.Init());
+  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 eariler (i.e. those threads do not call Init()).
+TEST_F(CertManagementTest, SignMultiThreadExclusiveSignleInit) {
+  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 eariler (i.e. those threads do not call Init()).
+TEST_F(CertManagementTest, SignMultiThreadSharedSignleInit) {
+  ASSERT_NO_FATAL_FAILURE(SignMultiThread(32, 16, SHARED, SINGLE_INIT));
+}
+
+// Generate CSR and issue corresponding certificate in a multi-thread fashion:
+// all threads use shared instances of CertRequestGenerator and CertSigner,
+// where every threads initializes those objects by itself.
+TEST_F(CertManagementTest, SignMultiThreadSharedMultiInit) {
+  ASSERT_NO_FATAL_FAILURE(SignMultiThread(16, 32, SHARED, MULTIPLE_INIT));
+}
+
+} // namespace ca
+} // namespace security
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/4a5f1392/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
new file mode 100644
index 0000000..15d4519
--- /dev/null
+++ b/src/kudu/security/test/test_certs.cc
@@ -0,0 +1,212 @@
+// 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/test/test_certs.h"
+
+namespace kudu {
+namespace security {
+namespace ca {
+
+//
+// The easiest way to create RSA private key and CA self-signed certificate pair
+// is using the couple of commands below:
+//
+//   openssl genrsa -out ca.pkey.pem 2048
+//   openssl req -new -x509 -batch -days 3650 -key ca.pkey.pem -out ca.cert.pem
+//
+// NOTE:
+//   The latter command uses configuration properties from default configuration
+//   file of the OpenSSL library.  Also, it runs in batch mode due to the
+//   '-batch' flag. To specify custom certificate subject properties, omit
+//   the '-batch' flag and run the command in interactive mode. If more
+//   customization is needed, see the other methods below.
+//
+////////////////////////////////////////////////////////////////////////////
+//
+// The other way to create RSA private key and CA self-signed certificate pair
+// is using OpenSSL's CA.sh script in $OPENSSL_SRC_ROOT/apps:
+//
+//   cp $OPENSSL_SRC_ROOT/CA.sh .
+//   chmod +x CA.sh
+//   ./CA.sh -newca
+//
+// Find the newly generated files at the following locations:
+//   * demoCA/cacert.pem:         self-signed CA certificate
+//   * demoCA/private/cakey.pem:  encrypted CA private key
+//
+// To decrypt the generated private key, run the following command and provide
+// the pass phrase (assuming that was an RSA key):
+//
+//   openssl rsa -in ./demoCA/private/cakey.pem
+//
+////////////////////////////////////////////////////////////////////////////
+//
+// Besides, the following sequence of commands can used to create
+// a private key and CA certficate with custom properties.
+//
+//  * Create a separate directory, e.g.:
+//
+//      mkdir /tmp/cert && cd /tmp/cert
+//
+//  * Create custom my.cnf configuration file for the OpenSSL library, copying
+//    the default one and modifying the result, if necessary.
+//
+//      cp $OPENSSL_CFG_ROOT/etc/openssl.cnf my.cnf
+//      vim my.cnf
+//
+//  * Create the CA directory structure which matches the directory structure
+//    of the 'default_ca' section from the configuration file, e.g.:
+//
+//      mkdir -p demoCA/certs demoCA/crl demoCA/newcerts demoCA/private
+//      touch demoCA/index.txt
+//
+//  * Create private key and certificate signing request (CSR):
+//
+//      openssl req -new -keyout ca.pkey.pem -out ca.req.pem \
+//        -subj "/C=US/ST=CA/O=MyCompany/CN=MyName/emailAddress=my@email.com" \
+//        -passin pass:mega_pass -passout pass:mega_pass -batch
+//
+//  * Create a self-signed certificate using the newly generated CSR as input:
+//
+//      openssl ca -config my.cnf -create_serial -days 3650 \
+//        -keyfile ca.pkey.pem -selfsign -extensions v3_ca \
+//        -outdir ./ -out ca.cert.pem -passin pass:mega_pass -batch \
+//        -infiles ca.req.pem
+//
+// The encryped private key is in ca.pkey.pem, the certificate is in
+// ca.cert.pem.  To decrypt the generated private key, execute the following
+// (assuming that was an RSA key):
+//
+//   openssl rsa -passin pass:mega_pass -in ./ca.pkey.pem
+//
+const char kCaCert[] = R"***(
+-----BEGIN CERTIFICATE-----
+MIIDizCCAnOgAwIBAgIJAIsQXjBhvdPoMA0GCSqGSIb3DQEBCwUAMFwxCzAJBgNV
+BAYTAlVTMQswCQYDVQQIDAJDQTESMBAGA1UECgwJTXlDb21wYW55MQ8wDQYDVQQD
+DAZNeU5hbWUxGzAZBgkqhkiG9w0BCQEWDG15QGVtYWlsLmNvbTAeFw0xNjEwMjUw
+NjAxNThaFw0yNjEwMjMwNjAxNThaMFwxCzAJBgNVBAYTAlVTMQswCQYDVQQIDAJD
+QTESMBAGA1UECgwJTXlDb21wYW55MQ8wDQYDVQQDDAZNeU5hbWUxGzAZBgkqhkiG
+9w0BCQEWDG15QGVtYWlsLmNvbTCCASIwDQYJKoZIhvcNAQEBBQADggEPADCCAQoC
+ggEBAKexXVOe0SfNexxl1nqMBRy8MCYWTl1kbRt5VQ698aXYcPNBC7gnEBW+8Yaa
+2f3Hl1Ye51zUGnOl4FU6HFDiIq59/lKCNG2X3amlYjzkImXn4M56r+5rEWs+HoHW
+kuqmMaxnrJatM86Of0K3j5QrOUft/qT5R6vSPnFH/pz+6ccBkAGV0UFVdshYSGkx
+KziVTdJ2Ri8oZgyeuReGxLkXOqKHzcOUFinvQ8fe8yaQr1kRAaPRo1eFqORXAMAU
+4KyvfiVjZMEGj0p47IekJHVPVVMopEmMMjhzRfbrxrKrMcIG6e4acF1KAd4wGI9A
+pCR3e1vcfbghDO7GhTMswLCnMYUCAwEAAaNQME4wHQYDVR0OBBYEFDc1+ybIwvG2
+IvEuAusZ9GGMlga/MB8GA1UdIwQYMBaAFDc1+ybIwvG2IvEuAusZ9GGMlga/MAwG
+A1UdEwQFMAMBAf8wDQYJKoZIhvcNAQELBQADggEBAJT9fL/vtayfAHpdzFvdWBe+
+R6y5HsVQQTBNF9x1eM6M0vGAlsXGgip3+RH7OMwurxNetL2mc03+PECas5LxB5Pr
+u1+kwtmv5YyfQzou0VwztjcbK2OEpWKj16XX6NO403iKoRF4fLn0DjZQcB0oXw4s
+vBxhNfz+SAsjsAMNgLHHXonJfg7wcdmNSp2N3TslGL/DH0bXMhsKx2CuMA3rd9WZ
+mJjItRIk8qNjazlmG0KYxQclP3lGagIMHxU6tY+iBXs1JR1/AUnPl/GaPeayCJSR
+3PB7R+MMrI0hfWFWkBt0D+UAKVa9to/N06wp4JqxEgOooU08PguXLIVDlW0xBcw=
+-----END CERTIFICATE-----
+)***";
+
+
+// See the comment for kCaCert_
+const char kCaPrivateKey[] = R"***(
+-----BEGIN RSA PRIVATE KEY-----
+MIIEogIBAAKCAQEAp7FdU57RJ817HGXWeowFHLwwJhZOXWRtG3lVDr3xpdhw80EL
+uCcQFb7xhprZ/ceXVh7nXNQac6XgVTocUOIirn3+UoI0bZfdqaViPOQiZefgznqv
+7msRaz4egdaS6qYxrGeslq0zzo5/QrePlCs5R+3+pPlHq9I+cUf+nP7pxwGQAZXR
+QVV2yFhIaTErOJVN0nZGLyhmDJ65F4bEuRc6oofNw5QWKe9Dx97zJpCvWREBo9Gj
+V4Wo5FcAwBTgrK9+JWNkwQaPSnjsh6QkdU9VUyikSYwyOHNF9uvGsqsxwgbp7hpw
+XUoB3jAYj0CkJHd7W9x9uCEM7saFMyzAsKcxhQIDAQABAoIBABuZQ0TZ5I5qcRKR
+aCUvGkBKcJo0HZ2dQ5+77lXIyRaEcsJ2OBmMxEbv8Aw5PBtaV/vihi1u8xOJf0xH
+jhV5wj95mPu3Vi2bSu36vBpNaaPf783Lv1y73lgKFzdDO1bHF3HKdksuIlKifStb
+zpOSMZE3CCvaowMSTRiTwsHP6mXIBdQ/TwAZHqGVTWDVGxc8JvoJ/3GjSgUIPKzy
+I2aS/5DQ+zmLktuP61GFMJg9tCSrwZPDi/XAatpoAOC9eA7AqF/l1TiaXsQN95mr
+mz2DkCoWRzAuDbya2Sh6nTJvpOMPAeXJ/MMZh9TWswJc4OAO2kZZsFfd0H6M1TKy
+1eAYKVkCgYEA1JhkKQ2h4cVzqQ9A5+4C0q5+j/RFDUOVnNlIjQiM73RchNu713mK
+zzhsom9S/6ZU8OH3TxzD54i2hHtX+QIJqVG0412QgAqAqnAKXGGkkAXiXGfGZhEW
+UB3OuTMbhfVqrkpj0wAPiEJAAuek7zES2B+gURUC24aAfOWU8xMkSjMCgYEAye4U
+e0NQ4HhhWRgWbgFYeAzsC/ezvlx30JjXiLPCNXGoLLJUCMjqWCPGYUvDonIJbxbj
++MYFkvYSDFGwTobKsB7FyT8DxPNus40zOh47y8QUK7jTL4nAmnBa3W9Oj00ceKpo
+wKe/adc2xPrS7mnVpz3ZkJ4I9z/MbEinyV5UTWcCgYAy8gXmlJ67dM6/r6kVK0M/
+65Lmulml0RFUUfmB2o+zfkYBjIqaG0U5XUMjNdxE6T4nr27NZY5IuMlMPCabxHI+
+Qhc/+Rb8qAenUEwbUUbXQKG7FR9FLEkVj98PIIEy+9nBxI/ha31NYNroF0y+CRuD
+8ShA5fEWXEgEJhwol+i1YwKBgEnGeiUuyvW4BZkPe+JlC3WRAwy8SydZkUzdCqIf
+Su1LwS3TWXB8N2JMb8ZMcAWBtICp1FCnyJGQ5bcqgUevZ45BL/H+29mxNtjS1cx+
+D0q7MMNom3/azEugkRImAIXKnoRXfj4lC4IX5yLAoSAJ+s1Hg52an5v16zIEuYiQ
+tiwxAoGAOP8/yjMzit1hzk27k9IfQSLD+1SqKCsRdGbAIhFRFlz4RUQOly1dEX8M
+qVmStlQ7N5gQWJSyDTe6rTe8pG9r030kNDJ+etr2KWpATGNaVWSmLWSYBXrPtejK
+gmbcYCewtt7dFP9tvx6k7aUQ6CKzg0GxaIHQecNzjxYrw8sb4Js=
+-----END RSA PRIVATE KEY-----
+)***";
+
+// See the comment for kCaCert_
+// (but use '-1' as number of days for the certificate expiration).
+const char kCaExpiredCert[] = R"***(
+-----BEGIN CERTIFICATE-----
+MIIDjTCCAnWgAwIBAgIJALNJes+nGWH9MA0GCSqGSIb3DQEBCwUAMF0xCzAJBgNV
+BAYTAlVTMQswCQYDVQQIDAJDQTETMBEGA1UECgwKRXhwQ29tcGFueTEQMA4GA1UE
+AwwHRXhwTmFtZTEaMBgGCSqGSIb3DQEJARYLZXhwQGV4cC5jb20wHhcNMTYxMDI1
+MTkzOTM4WhcNMTYxMDI0MTkzOTM4WjBdMQswCQYDVQQGEwJVUzELMAkGA1UECAwC
+Q0ExEzARBgNVBAoMCkV4cENvbXBhbnkxEDAOBgNVBAMMB0V4cE5hbWUxGjAYBgkq
+hkiG9w0BCQEWC2V4cEBleHAuY29tMIIBIjANBgkqhkiG9w0BAQEFAAOCAQ8AMIIB
+CgKCAQEAzqPj5nRm57mr9YtZDvHREuVFHTwPcKzDeff9fnrKKwOJPSF0Bou/BjS1
+S7yQYAtmT/EMi7qxEWjgrR1qW+muR8QN+zAwNdkdLrFK3SJigQ4a/OeSH86aHXUD
+ekV8mgBgzP90osbHf7AiqrGzkYWq+ApTO/IgnXgaWbbdt5znGTW5lKQ4O2CYhpcM
+MC1sBBjW7Qqx+Gi8iXub0zlJ2mVI8o+zb9qvSDb8fa0JYxasRDn/nB0wKZC3f/Gf
+Rs+lJZUTEy5+eMhVdj1RjVBE+mgW7L27On24ViPU7B3DjM0SYnD6ZOUWMH0mtwO8
+W3OoK8MJhPvFP7Lr5QfSjiBH+ryLOwIDAQABo1AwTjAdBgNVHQ4EFgQUsp8OZLl1
+2Z/2aXBQRH0Z+nWxqXcwHwYDVR0jBBgwFoAUsp8OZLl12Z/2aXBQRH0Z+nWxqXcw
+DAYDVR0TBAUwAwEB/zANBgkqhkiG9w0BAQsFAAOCAQEArWvFi13iqmvnY0xkgt3R
+acurTvWcQzcUOgVPF8u1atj9d+0zrMk7Don1KQp6uYLdeNLL8NbL4oLxtidW/Yap
+ZEbHVDQTeZAsT7Hr+0uD3vMUndsjG7C85tOhZMiGukFPhuaHE5KmQEy6nUCaJiAv
+opZlNj1mEOGyshSXHsBATl9o33WLTLfPqrO3/12jExApHiADcON4RsPUV6M6k5A2
+/KghYEPYAuFfXTsqj+W7HRL1UuiHJxW96ySQqYzQ86aRN2ZZlTdbDnIU5Jrb6YJB
+hUALcxIUhtodui61zsJFIkVauxTxk7jNCwRvj4I1dSSFWA63t9eh7sKilLRCayNl
+yQ==
+-----END CERTIFICATE-----
+)***";
+
+// See the comment for kCaExpiredCert_
+const char kCaExpiredPrivateKey[] = R"***(
+-----BEGIN RSA PRIVATE KEY-----
+MIIEpAIBAAKCAQEAzqPj5nRm57mr9YtZDvHREuVFHTwPcKzDeff9fnrKKwOJPSF0
+Bou/BjS1S7yQYAtmT/EMi7qxEWjgrR1qW+muR8QN+zAwNdkdLrFK3SJigQ4a/OeS
+H86aHXUDekV8mgBgzP90osbHf7AiqrGzkYWq+ApTO/IgnXgaWbbdt5znGTW5lKQ4
+O2CYhpcMMC1sBBjW7Qqx+Gi8iXub0zlJ2mVI8o+zb9qvSDb8fa0JYxasRDn/nB0w
+KZC3f/GfRs+lJZUTEy5+eMhVdj1RjVBE+mgW7L27On24ViPU7B3DjM0SYnD6ZOUW
+MH0mtwO8W3OoK8MJhPvFP7Lr5QfSjiBH+ryLOwIDAQABAoIBABszgcWNXxpz24oI
+HOIVvPLi0VVG2bV4WIcOuQTUPxaocYFljPNro+q6N39PxCWQephdX8xo9/QVvTWs
+oJqWyUVTLo/5SO9dtDS4S+WOKC9a3vyZsyeSt8DW7W1EBmHzWMrDeeQPjKVnVzjn
+CX9HfDkIiupiNh7kd3uF0evgsJ8lsZ65HtBq9MWu+mIR1H0EpRLxywdoRJLJ+JdW
+g1fLFRuhnWo0GcEyBK45kLCoVJsRbCkFGf6uPDOOC0g5mIyxGclWeF6ps1OFnFyu
+FWsYeMLSt5tYZfB0/QR46X9HQOhfLunjA04VBkScSRjlohGO4d20ZW7HlPY20CbR
+1PHhEvkCgYEA98FYoovNezx8OgkcAtNOOTK7GpUaUfh3Xl5yPGgCqxoG8G+BTmKF
+MGlIf6URKQA0BUtNdjIvfIcaIctj56qFwjHL6CbzR5MkXUZLlyl0XzYFXm/lavr4
+Z5DHWdFo+GyFaiXIiVof93jAnOFgjSxdhHaEhQqj7pmaBoHVZqtwHFcCgYEA1YRH
+xTzcWErp06KJTt+/P4YtWRh9GDBhhlO3oaGOANkEab8cGjRO9LJP24wyo7exXqGb
+UjtEifEHtzhj6a/UwSAMsFcNhlQRvy525HD1gJmQ2m4wZ3GxztK4IZ4rVDjsB5/D
+SMMBsDfs1r1iRwdSMHAOhrVH2l/DMFQLnx1x+b0CgYEAlQm6SA3RjlDUahUQxKJY
+bBAYfeUz8BuHsz0dezkWYddGVVy+bGjXtkefVSn3KLL2mDi0YGXQKxkanzm636G0
+1R0fjIfh0Syys2mWD1jgqGXW1Ph7Cd/vjl2Jjn5qpwahOzl/aSDOGhCJzdXGPyZx
+Gz4wedfsxZuhDEkOFrUKvAECgYEAxHYYy8V6Qct8Z30wtmBuSvcdFtPPlsg9lCnH
+13MdhG4q/1oXc40Z8VF45VyU48uL6rTsg7eBEyOyo8XBOS7Opnzk8ATJrwX/5lfM
+kdnWK2QhwrqM00HsB5AgWN5+o9pUY5d/Sp4UGZ77z4MmwJBd8a/Jze1Tlf1zTi6n
+GtsvGkkCgYAfILUAPf+ujgB9zdsJa+4l9XCEq0j39/Usfj0VrInNAk7RN8W0qNw7
+ZLs3Qt2fgPO0CeMeVUVKcvdjlXq3EbrWKrsJLxy3Gb8ruBjIlJqncJn6mKslXS+l
+H/sbP2R+P6RvQceLEEtk6ZZLiuScVmLtVOpUoUZb3Rx6a7GKbec7oQ==
+-----END RSA PRIVATE KEY-----
+)***";
+
+} // namespace ca
+} // namespace security
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/4a5f1392/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
new file mode 100644
index 0000000..bafc11b
--- /dev/null
+++ b/src/kudu/security/test/test_certs.h
@@ -0,0 +1,45 @@
+// 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
+
+namespace kudu {
+namespace security {
+namespace ca {
+
+//
+// Set of certificates and private keys used for certificate generation
+// and signing tests (declarations).  See the .cc file for the actual data.
+//
+
+// Valid root CA cerificate (PEM format).
+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[];
+
+// 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[];
+
+} // namespace ca
+} // namespace security
+} // namespace kudu
+