You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by sz...@apache.org on 2022/05/25 11:37:24 UTC

[nifi-minifi-cpp] 01/02: MINIFICPP-1826 Log warning on certificate that is about to expire

This is an automated email from the ASF dual-hosted git repository.

szaszm pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/nifi-minifi-cpp.git

commit d95e9201cb9faa5998c7e36038ed90df6dd992e5
Author: Adam Debreceni <ad...@apache.org>
AuthorDate: Wed May 25 13:11:21 2022 +0200

    MINIFICPP-1826 Log warning on certificate that is about to expire
    
    Closes #1336
    Signed-off-by: Marton Szasz <sz...@apache.org>
---
 extensions/http-curl/tests/VerifyInvokeHTTP.h     |   2 +
 libminifi/include/controllers/SSLContextService.h |  13 +-
 libminifi/include/utils/tls/CertificateUtils.h    |  63 +++++
 libminifi/src/controllers/SSLContextService.cpp   | 272 ++++++++++++++--------
 libminifi/src/utils/tls/CertificateUtils.cpp      | 149 ++++++++++++
 5 files changed, 401 insertions(+), 98 deletions(-)

diff --git a/extensions/http-curl/tests/VerifyInvokeHTTP.h b/extensions/http-curl/tests/VerifyInvokeHTTP.h
index 92687eaf1..ec1ad6a5a 100644
--- a/extensions/http-curl/tests/VerifyInvokeHTTP.h
+++ b/extensions/http-curl/tests/VerifyInvokeHTTP.h
@@ -28,6 +28,7 @@
 #include "HTTPClient.h"
 #include "InvokeHTTP.h"
 #include "processors/LogAttribute.h"
+#include "controllers/SSLContextService.h"
 #include "core/state/ProcessorController.h"
 #include "HTTPIntegrationBase.h"
 
@@ -46,6 +47,7 @@ class VerifyInvokeHTTP : public HTTPIntegrationBase {
     LogTestController::getInstance().setTrace<minifi::processors::LogAttribute>();
     LogTestController::getInstance().setDebug<core::Processor>();
     LogTestController::getInstance().setDebug<core::ProcessSession>();
+    LogTestController::getInstance().setDebug<minifi::controllers::SSLContextService>();
   }
 
   void setUrl(const std::string &url, ServerAwareHandler *handler) override {
diff --git a/libminifi/include/controllers/SSLContextService.h b/libminifi/include/controllers/SSLContextService.h
index a3ef480b1..d75d8f370 100644
--- a/libminifi/include/controllers/SSLContextService.h
+++ b/libminifi/include/controllers/SSLContextService.h
@@ -42,6 +42,7 @@
 #include "../core/controller/ControllerService.h"
 #include "core/logging/LoggerConfiguration.h"
 #include "utils/Export.h"
+#include "utils/tls/CertificateUtils.h"
 
 namespace org {
 namespace apache {
@@ -228,11 +229,19 @@ class SSLContextService : public core::controller::ControllerService {
   bool addClientCertificateFromSystemStoreToSSLContext(SSL_CTX* ctx) const;
   bool addServerCertificatesFromSystemStoreToSSLContext(SSL_CTX* ctx) const;
 #ifdef WIN32
-  bool useClientCertificate(SSL_CTX* ctx, PCCERT_CONTEXT certificate) const;
-  void addServerCertificateToSSLStore(X509_STORE* ssl_store, PCCERT_CONTEXT certificate) const;
+  using ClientCertCallback = std::function<bool(utils::tls::X509_unique_ptr cert, utils::tls::EVP_PKEY_unique_ptr priv_key)>;
+  using ServerCertCallback = std::function<bool(utils::tls::X509_unique_ptr cert)>;
+
+  bool findClientCertificate(ClientCertCallback cb) const;
+  bool findServerCertificate(ServerCertCallback cb) const;
+
+  bool useClientCertificate(PCCERT_CONTEXT certificate, ClientCertCallback cb) const;
+  bool useServerCertificate(PCCERT_CONTEXT certificate, ServerCertCallback cb) const;
 #endif  // WIN32
 #endif  // OPENSSL_SUPPORT
 
+  void verifyCertificateExpiration();
+
   std::shared_ptr<core::logging::Logger> logger_;
 };
 typedef int (SSLContextService::*ptr)(char *, int, int, void *);
diff --git a/libminifi/include/utils/tls/CertificateUtils.h b/libminifi/include/utils/tls/CertificateUtils.h
index bb4d467f6..73e7011fc 100644
--- a/libminifi/include/utils/tls/CertificateUtils.h
+++ b/libminifi/include/utils/tls/CertificateUtils.h
@@ -18,6 +18,7 @@
 #ifdef OPENSSL_SUPPORT
 
 #include <openssl/ssl.h>
+#include <openssl/pkcs12.h>
 
 #ifdef WIN32
 #include <windows.h>
@@ -25,6 +26,12 @@
 #endif  // WIN32
 
 #include <memory>
+#include <filesystem>
+#include <optional>
+#include <string>
+#include <functional>
+
+#include "WindowsCertStoreLocation.h"
 
 namespace org {
 namespace apache {
@@ -33,6 +40,22 @@ namespace minifi {
 namespace utils {
 namespace tls {
 
+class ssl_error_category : public std::error_category {
+ public:
+  [[nodiscard]]
+  const char* name() const noexcept override {
+    return "ssl_error";
+  }
+
+  [[nodiscard]]
+  static const ssl_error_category& get();
+
+  [[nodiscard]]
+  std::string message(int value) const override;
+};
+
+std::error_code get_last_ssl_error_code();
+
 struct EVP_PKEY_deleter {
   void operator()(EVP_PKEY* pkey) const { EVP_PKEY_free(pkey); }
 };
@@ -43,7 +66,33 @@ struct X509_deleter {
 };
 using X509_unique_ptr = std::unique_ptr<X509, X509_deleter>;
 
+struct BIO_deleter {
+  void operator()(BIO* bio) const  { BIO_free(bio); }
+};
+using BIO_unique_ptr = std::unique_ptr<BIO, BIO_deleter>;
+
+struct PKCS12_deleter {
+  void operator()(PKCS12* cert) const  { PKCS12_free(cert); }
+};
+using PKCS12_unique_ptr = std::unique_ptr<PKCS12, PKCS12_deleter>;
+
 #ifdef WIN32
+class WindowsCertStore {
+ public:
+  WindowsCertStore(const WindowsCertStoreLocation& loc, const std::string& cert_store);
+
+  std::error_code error() const;
+
+  PCCERT_CONTEXT nextCert();
+
+  ~WindowsCertStore();
+
+ private:
+  std::error_code error_;
+  HCERTSTORE store_ptr_;
+  PCCERT_CONTEXT cert_ctx_ptr_ = nullptr;
+};
+
 // Returns nullptr on errors
 X509_unique_ptr convertWindowsCertificate(PCCERT_CONTEXT certificate);
 
@@ -51,6 +100,20 @@ X509_unique_ptr convertWindowsCertificate(PCCERT_CONTEXT certificate);
 EVP_PKEY_unique_ptr extractPrivateKey(PCCERT_CONTEXT certificate);
 #endif  // WIN32
 
+std::string getLatestOpenSSLErrorString();
+
+std::optional<std::chrono::system_clock::time_point> getCertificateExpiration(const X509_unique_ptr& cert);
+
+struct CertHandler {
+  std::function<std::error_code(X509_unique_ptr cert)> cert_cb;
+  std::function<std::error_code(X509_unique_ptr cert)> chain_cert_cb;
+  std::function<std::error_code(EVP_PKEY_unique_ptr priv_key)> priv_key_cb;
+};
+
+std::error_code processP12Certificate(const std::string& cert_file, const std::string& passphrase, const CertHandler& handler);
+
+std::error_code processPEMCertificate(const std::string& cert_file, const std::optional<std::string>& passphrase, const CertHandler& handler);
+
 }  // namespace tls
 }  // namespace utils
 }  // namespace minifi
diff --git a/libminifi/src/controllers/SSLContextService.cpp b/libminifi/src/controllers/SSLContextService.cpp
index e410a5ecd..5cb74e68b 100644
--- a/libminifi/src/controllers/SSLContextService.cpp
+++ b/libminifi/src/controllers/SSLContextService.cpp
@@ -43,6 +43,7 @@
 #include "utils/tls/TLSUtils.h"
 #include "utils/tls/DistinguishedName.h"
 #include "utils/tls/WindowsCertStoreLocation.h"
+#include "utils/TimeUtil.h"
 
 namespace org {
 namespace apache {
@@ -175,51 +176,31 @@ bool SSLContextService::configure_ssl_context(SSL_CTX *ctx) {
 }
 
 bool SSLContextService::addP12CertificateToSSLContext(SSL_CTX* ctx) const {
-  const auto fp_deleter = [](BIO* ptr) { BIO_free(ptr); };
-  std::unique_ptr<BIO, decltype(fp_deleter)> fp(BIO_new(BIO_s_file()), fp_deleter);
-  if (fp == nullptr) {
-    core::logging::LOG_ERROR(logger_) << "Failed create new file BIO, " << getLatestOpenSSLErrorString();
-    return false;
-  }
-  if (BIO_read_filename(fp.get(), certificate_.c_str()) <= 0) {
-    core::logging::LOG_ERROR(logger_) << "Failed to read certificate file " << certificate_ << ", " << getLatestOpenSSLErrorString();
-    return false;
-  }
-  const auto p12_deleter = [](PKCS12* ptr) { PKCS12_free(ptr); };
-  std::unique_ptr<PKCS12, decltype(p12_deleter)> p12(d2i_PKCS12_bio(fp.get(), nullptr), p12_deleter);
-  if (p12 == nullptr) {
-    core::logging::LOG_ERROR(logger_) << "Failed to DER decode certificate file " << certificate_ << ", " << getLatestOpenSSLErrorString();
-    return false;
-  }
-
-  EVP_PKEY* pkey = nullptr;
-  X509* cert = nullptr;
-  STACK_OF(X509)* ca = nullptr;
-  if (!PKCS12_parse(p12.get(), passphrase_.c_str(), &pkey, &cert, &ca)) {
-    core::logging::LOG_ERROR(logger_) << "Failed to parse certificate file " << certificate_ << " as PKCS#12, " << getLatestOpenSSLErrorString();
-    return false;
-  }
-  utils::tls::EVP_PKEY_unique_ptr pkey_ptr{pkey};
-  utils::tls::X509_unique_ptr cert_ptr{cert};
-  const auto ca_deleter = gsl::finally([ca] { sk_X509_pop_free(ca, X509_free); });
-
-  if (SSL_CTX_use_certificate(ctx, cert) != 1) {
-    core::logging::LOG_ERROR(logger_) << "Failed to set certificate from " << certificate_ << ", " << getLatestOpenSSLErrorString();
-    return false;
-  }
-  while (ca != nullptr && sk_X509_num(ca) > 0) {
-    utils::tls::X509_unique_ptr cacert{sk_X509_pop(ca)};
-    if (SSL_CTX_add_extra_chain_cert(ctx, cacert.get()) != 1) {
-      core::logging::LOG_ERROR(logger_) << "Failed to set additional certificate from " << certificate_ << ", " << getLatestOpenSSLErrorString();
-      return false;
+  auto error = utils::tls::processP12Certificate(certificate_, passphrase_, {
+    .cert_cb = [&] (auto cert) -> std::error_code {
+      if (SSL_CTX_use_certificate(ctx, cert.get()) != 1) {
+        return utils::tls::get_last_ssl_error_code();
+      }
+      return {};
+    },
+    .chain_cert_cb = [&] (auto cacert) -> std::error_code {
+      if (SSL_CTX_add_extra_chain_cert(ctx, cacert.get()) != 1) {
+        return utils::tls::get_last_ssl_error_code();
+      }
+      cacert.release();  // a successful SSL_CTX_add_extra_chain_cert() takes ownership of cacert
+      return {};
+    },
+    .priv_key_cb = [&] (auto priv_key) -> std::error_code {
+      if (SSL_CTX_use_PrivateKey(ctx, priv_key.get()) != 1) {
+        return utils::tls::get_last_ssl_error_code();
+      }
+      return {};
     }
-    cacert.release();  // a successful SSL_CTX_add_extra_chain_cert() takes ownership of cacert
-  }
-  if (SSL_CTX_use_PrivateKey(ctx, pkey) != 1) {
-    core::logging::LOG_ERROR(logger_) << "Failed to set private key from " << certificate_ << ", " << getLatestOpenSSLErrorString();
+  });
+  if (error) {
+    core::logging::LOG_ERROR(logger_) << error.message();
     return false;
   }
-
   return true;
 }
 
@@ -247,24 +228,17 @@ bool SSLContextService::addPemCertificateToSSLContext(SSL_CTX* ctx) const {
 }
 
 #ifdef WIN32
-bool SSLContextService::addClientCertificateFromSystemStoreToSSLContext(SSL_CTX* ctx) const {
-  utils::tls::WindowsCertStoreLocation store_location{cert_store_location_};
-  HCERTSTORE hCertStore = CertOpenStore(CERT_STORE_PROV_SYSTEM_A, 0, NULL,
-                                        CERT_STORE_OPEN_EXISTING_FLAG | CERT_STORE_READONLY_FLAG | store_location.getBitfieldValue(),
-                                        client_cert_store_.data());
-  if (!hCertStore) {
-    logger_->log_error("Could not open system certificate store %s/%s (client certificates)", cert_store_location_, client_cert_store_);
+bool SSLContextService::findClientCertificate(ClientCertCallback cb) const {
+  utils::tls::WindowsCertStore cert_store(utils::tls::WindowsCertStoreLocation{cert_store_location_}, client_cert_store_);
+  if (auto error = cert_store.error()) {
+    logger_->log_error("Could not open system certificate store %s/%s (client certificates): %s", cert_store_location_, client_cert_store_, error.message());
     return false;
   }
-  const auto store_close = gsl::finally([hCertStore](){ CertCloseStore(hCertStore, 0); });
 
   logger_->log_debug("Looking for client certificate in sytem store %s/%s", cert_store_location_, client_cert_store_);
 
-  PCCERT_CONTEXT pCertContext = nullptr;
-  while (pCertContext = CertEnumCertificatesInStore(hCertStore, pCertContext)) {
-    bool certificateIsAcceptableAndWasSuccessfullyAddedToSSLContext = useClientCertificate(ctx, pCertContext);
-    if (certificateIsAcceptableAndWasSuccessfullyAddedToSSLContext) {
-      CertFreeCertificateContext(pCertContext);
+  while (auto cert_ctx = cert_store.nextCert()) {
+    if (useClientCertificate(cert_ctx, cb)) {
       return true;
     }
   }
@@ -272,6 +246,24 @@ bool SSLContextService::addClientCertificateFromSystemStoreToSSLContext(SSL_CTX*
   logger_->log_error("Could not find any suitable client certificate in sytem store %s/%s", cert_store_location_, client_cert_store_);
   return false;
 }
+
+#endif
+
+#ifdef WIN32
+bool SSLContextService::addClientCertificateFromSystemStoreToSSLContext(SSL_CTX* ctx) const {
+  return findClientCertificate([&] (auto cert, auto priv_key) -> bool {
+    if (SSL_CTX_use_certificate(ctx, cert.get()) != 1) {
+      logger_->log_error("Failed to set certificate from %s, %s", cert->name, getLatestOpenSSLErrorString);
+      return false;
+    }
+
+    if (SSL_CTX_use_PrivateKey(ctx, priv_key.get()) != 1) {
+      logger_->log_error("Failed to use private key %s, %s", cert->name, getLatestOpenSSLErrorString());
+      return false;
+    }
+    return true;
+  });
+}
 #else
 bool SSLContextService::addClientCertificateFromSystemStoreToSSLContext(SSL_CTX* /*ctx*/) const {
   logger_->log_error("Getting client certificate from the system store is only supported on Windows");
@@ -280,7 +272,7 @@ bool SSLContextService::addClientCertificateFromSystemStoreToSSLContext(SSL_CTX*
 #endif  // WIN32
 
 #ifdef WIN32
-bool SSLContextService::useClientCertificate(SSL_CTX* ctx, PCCERT_CONTEXT certificate) const {
+bool SSLContextService::useClientCertificate(PCCERT_CONTEXT certificate, ClientCertCallback cb) const {
   utils::tls::X509_unique_ptr x509_cert = utils::tls::convertWindowsCertificate(certificate);
   if (!x509_cert) {
     logger_->log_error("Failed to convert system store client certificate to X.509 format");
@@ -314,46 +306,41 @@ bool SSLContextService::useClientCertificate(SSL_CTX* ctx, PCCERT_CONTEXT certif
     return false;
   }
 
-  if (SSL_CTX_use_certificate(ctx, x509_cert.get()) != 1) {
-    logger_->log_error("Failed to set certificate from %s, %s", x509_cert->name, getLatestOpenSSLErrorString);
-    return false;
-  }
-
-  if (SSL_CTX_use_PrivateKey(ctx, private_key.get()) != 1) {
-    logger_->log_error("Failed to use private key %s, %s", x509_cert->name, getLatestOpenSSLErrorString());
-    return false;
+  std::string cert_name = x509_cert->name;
+  if (cb(std::move(x509_cert), std::move(private_key))) {
+    logger_->log_debug("Found client certificate %s", cert_name);
+    return true;
   }
 
-  logger_->log_debug("Found client certificate %s", x509_cert->name);
-
-  return true;
+  return false;
 }
 #endif  // WIN32
 
 bool SSLContextService::addServerCertificatesFromSystemStoreToSSLContext(SSL_CTX* ctx) const {
 #ifdef WIN32
-  utils::tls::WindowsCertStoreLocation store_location{cert_store_location_};
-  HCERTSTORE hCertStore = CertOpenStore(CERT_STORE_PROV_SYSTEM_A, 0, NULL,
-                                        CERT_STORE_OPEN_EXISTING_FLAG | CERT_STORE_READONLY_FLAG | store_location.getBitfieldValue(),
-                                        server_cert_store_.data());
-  if (!hCertStore) {
-    logger_->log_error("Could not open system certificate store %s/%s (server certificates)", cert_store_location_, server_cert_store_);
-    return false;
-  }
-  const auto store_close = gsl::finally([hCertStore](){ CertCloseStore(hCertStore, 0); });
-
   X509_STORE* ssl_store = SSL_CTX_get_cert_store(ctx);
   if (!ssl_store) {
     logger_->log_error("Could not get handle to SSL certificate store");
     return false;
   }
 
-  logger_->log_debug("Adding server certificates from system store %s/%s", cert_store_location_, server_cert_store_);
+  findServerCertificate([&] (auto cert) -> bool {
+    // return false to indicate that we wish to iterate over all subsequent certificates as well
+    int success = X509_STORE_add_cert(ssl_store, cert.get());
+    if (success == 1) {
+      logger_->log_debug("Added server certificate %s from the system store to the SSL store", cert->name);
+      return false;
+    }
 
-  PCCERT_CONTEXT pCertContext = nullptr;
-  while (pCertContext = CertEnumCertificatesInStore(hCertStore, pCertContext)) {
-    addServerCertificateToSSLStore(ssl_store, pCertContext);
-  }
+    auto err = ERR_peek_last_error();
+    if (ERR_GET_REASON(err) == X509_R_CERT_ALREADY_IN_HASH_TABLE) {
+      logger_->log_debug("Ignoring duplicate server certificate %s", cert->name);
+      return false;
+    }
+
+    logger_->log_error("Failed to add server certificate %s to the SSL store; error: %s", cert->name, getLatestOpenSSLErrorString());
+    return false;
+  });
 
   return true;
 #else
@@ -363,26 +350,34 @@ bool SSLContextService::addServerCertificatesFromSystemStoreToSSLContext(SSL_CTX
 }
 
 #ifdef WIN32
-void SSLContextService::addServerCertificateToSSLStore(X509_STORE* ssl_store, PCCERT_CONTEXT certificate) const {
-  utils::tls::X509_unique_ptr x509_cert = utils::tls::convertWindowsCertificate(certificate);
-  if (!x509_cert) {
-    logger_->log_error("Failed to convert system store server certificate to X.509 format");
-    return;
+bool SSLContextService::findServerCertificate(ServerCertCallback cb) const {
+  utils::tls::WindowsCertStore cert_store(utils::tls::WindowsCertStoreLocation{cert_store_location_}, server_cert_store_);
+  if (auto error = cert_store.error()) {
+    logger_->log_error("Could not open system certificate store %s/%s (server certificates): %s", cert_store_location_, server_cert_store_, error.message());
+    return false;
   }
 
-  int success = X509_STORE_add_cert(ssl_store, x509_cert.get());
-  if (success == 1) {
-    logger_->log_debug("Added server certificate %s from the system store to the SSL store", x509_cert->name);
-    return;
+  logger_->log_debug("Adding server certificates from system store %s/%s", cert_store_location_, server_cert_store_);
+
+  while (auto cert_ctx = cert_store.nextCert()) {
+    if (useServerCertificate(cert_ctx, cb)) {
+      return true;
+    }
   }
 
-  auto err = ERR_peek_last_error();
-  if (ERR_GET_REASON(err) == X509_R_CERT_ALREADY_IN_HASH_TABLE) {
-    logger_->log_debug("Ignoring duplicate server certificate %s", x509_cert->name);
-    return;
+  return false;
+}
+#endif
+
+#ifdef WIN32
+bool SSLContextService::useServerCertificate(PCCERT_CONTEXT certificate, ServerCertCallback cb) const {
+  utils::tls::X509_unique_ptr x509_cert = utils::tls::convertWindowsCertificate(certificate);
+  if (!x509_cert) {
+    logger_->log_error("Failed to convert system store server certificate to X.509 format");
+    return false;
   }
 
-  logger_->log_error("Failed to add server certificate %s to the SSL store; error: %s", x509_cert->name, getLatestOpenSSLErrorString());
+  return cb(std::move(x509_cert));
 }
 #endif  // WIN32
 #endif  // OPENSSL_SUPPORT
@@ -544,6 +539,8 @@ void SSLContextService::onEnable() {
   getProperty(ClientCertKeyUsage.getName(), client_cert_key_usage);
   client_cert_key_usage_ = utils::tls::ExtendedKeyUsage{client_cert_key_usage};
 #endif  // WIN32
+
+  verifyCertificateExpiration();
 }
 
 void SSLContextService::initializeProperties() {
@@ -563,6 +560,89 @@ void SSLContextService::initializeProperties() {
   setSupportedProperties(supportedProperties);
 }
 
+void SSLContextService::verifyCertificateExpiration() {
+  auto verify = [&] (const std::string& cert_file, const utils::tls::X509_unique_ptr& cert) {
+    if (auto end_date = utils::tls::getCertificateExpiration(cert)) {
+      std::string end_date_str = getTimeStr(std::chrono::duration_cast<std::chrono::milliseconds>(end_date->time_since_epoch()).count());
+      if (end_date.value() < std::chrono::system_clock::now()) {
+        core::logging::LOG_ERROR(logger_) << "Certificate in '" << cert_file << "' expired at " << end_date_str;
+      } else if (auto diff = end_date.value() - std::chrono::system_clock::now(); diff < std::chrono::weeks{2}) {
+        core::logging::LOG_WARN(logger_) << "Certificate in '" << cert_file << "' will expire at " << end_date_str;
+      } else {
+        core::logging::LOG_DEBUG(logger_) << "Certificate in '" << cert_file << "' will expire at " << end_date_str;
+      }
+    } else {
+      core::logging::LOG_ERROR(logger_) << "Could not determine expiration date for certificate in '" << cert_file << "'";
+    }
+  };
+  if (!IsNullOrEmpty(certificate_)) {
+    if (isFileTypeP12(certificate_)) {
+      auto error = utils::tls::processP12Certificate(certificate_, passphrase_, {
+          .cert_cb = [&](auto cert) -> std::error_code {
+            verify(certificate_, cert);
+            return {};
+          },
+          .chain_cert_cb = [&](auto cert) -> std::error_code {
+            verify(certificate_, cert);
+            return {};
+          },
+          .priv_key_cb = {}
+      });
+      if (error) {
+        core::logging::LOG_ERROR(logger_) << error.value();
+      }
+    } else {
+      auto error = utils::tls::processPEMCertificate(certificate_, passphrase_, {
+          .cert_cb = [&](auto cert) -> std::error_code {
+            verify(certificate_, cert);
+            return {};
+          },
+          .chain_cert_cb = [&](auto cert) -> std::error_code {
+            verify(certificate_, cert);
+            return {};
+          },
+          .priv_key_cb = {}
+      });
+      if (error) {
+        core::logging::LOG_ERROR(logger_) << error.value();
+      }
+    }
+  }
+
+  if (!IsNullOrEmpty(ca_certificate_)) {
+    auto error = utils::tls::processPEMCertificate(ca_certificate_, std::nullopt, {
+        .cert_cb = [&](auto cert) -> std::error_code {
+          verify(ca_certificate_, cert);
+          return {};
+        },
+        .chain_cert_cb = [&](auto cert) -> std::error_code {
+          verify(ca_certificate_, cert);
+          return {};
+        },
+        .priv_key_cb = {}
+    });
+    if (error) {
+      core::logging::LOG_ERROR(logger_) << error.message();
+    }
+  }
+
+#ifdef WIN32
+  if (use_system_cert_store_ && IsNullOrEmpty(certificate_)) {
+    findClientCertificate([&] (auto cert, auto /*priv_key*/) -> bool {
+      verify(cert->name, cert);
+      return false;  // keep on iterating, check all
+    });
+  }
+
+  if (use_system_cert_store_ && IsNullOrEmpty(ca_certificate_)) {
+    findServerCertificate([&] (auto cert) -> bool {
+      verify(cert->name, cert);
+      return false;  // keep on iterating, check all
+    });
+  }
+#endif
+}
+
 REGISTER_RESOURCE(SSLContextService, "Controller service that provides SSL/TLS capabilities to consuming interfaces");
 
 } /* namespace controllers */
diff --git a/libminifi/src/utils/tls/CertificateUtils.cpp b/libminifi/src/utils/tls/CertificateUtils.cpp
index ee76c24fc..6fe4fe1bf 100644
--- a/libminifi/src/utils/tls/CertificateUtils.cpp
+++ b/libminifi/src/utils/tls/CertificateUtils.cpp
@@ -19,12 +19,17 @@
 #include "utils/tls/CertificateUtils.h"
 
 #include <openssl/rsa.h>
+#include <openssl/err.h>
 
 #ifdef WIN32
 #pragma comment(lib, "ncrypt.lib")
 #pragma comment(lib, "Ws2_32.lib")
 #endif  // WIN32
 
+#include "utils/StringUtils.h"
+#include "utils/tls/TLSUtils.h"
+#include "utils/TimeUtil.h"
+
 namespace org {
 namespace apache {
 namespace nifi {
@@ -32,7 +37,53 @@ namespace minifi {
 namespace utils {
 namespace tls {
 
+const ssl_error_category& ssl_error_category::get() {
+  static ssl_error_category instance;
+  return instance;
+}
+
+std::string ssl_error_category::message(int value) const {
+  auto err = gsl::narrow<unsigned long>(value);  // NOLINT
+  if (err == 0) {
+    return "";
+  }
+  char buf[4096];
+  ERR_error_string_n(err, buf, sizeof(buf));
+  return buf;
+}
+
+std::error_code get_last_ssl_error_code() {
+  return std::error_code{gsl::narrow<int>(ERR_peek_last_error()), ssl_error_category::get()};
+}
+
 #ifdef WIN32
+WindowsCertStore::WindowsCertStore(const WindowsCertStoreLocation& loc, const std::string& cert_store) {
+  store_ptr_ = CertOpenStore(CERT_STORE_PROV_SYSTEM_A, 0, NULL,
+                             CERT_STORE_OPEN_EXISTING_FLAG | CERT_STORE_READONLY_FLAG | loc.getBitfieldValue(),
+                             cert_store.data());
+
+  if (!store_ptr_) {
+    error_ = std::error_code{WSAGetLastError(), std::system_category()};
+  }
+}
+
+std::error_code WindowsCertStore::error() const {
+  return error_;
+}
+
+PCCERT_CONTEXT WindowsCertStore::nextCert() {
+  return cert_ctx_ptr_ = CertEnumCertificatesInStore(store_ptr_, cert_ctx_ptr_);
+}
+
+WindowsCertStore::~WindowsCertStore() {
+  if (cert_ctx_ptr_) {
+    CertFreeCertificateContext(cert_ctx_ptr_);
+  }
+  if (store_ptr_) {
+    CertCloseStore(store_ptr_, 0);
+  }
+}
+
 X509_unique_ptr convertWindowsCertificate(const PCCERT_CONTEXT certificate) {
   const unsigned char *certificate_binary = certificate->pbCertEncoded;
   long certificate_length = certificate->cbCertEncoded;  // NOLINT: cpplint hates `long`, but that is the param type in the API
@@ -116,6 +167,104 @@ EVP_PKEY_unique_ptr extractPrivateKey(const PCCERT_CONTEXT certificate) {
 }
 #endif  // WIN32
 
+std::string getLatestOpenSSLErrorString() {
+  return get_last_ssl_error_code().message();
+}
+
+std::optional<std::chrono::system_clock::time_point> getCertificateExpiration(const X509_unique_ptr& cert) {
+  const ASN1_TIME* asn1_end = X509_get0_notAfter(cert.get());
+  if (!asn1_end) {
+    return {};
+  }
+  std::tm end{};
+  int ret = ASN1_time_parse(reinterpret_cast<const char*>(asn1_end->data), asn1_end->length, &end, 0);
+  if (ret == -1) {
+    return {};
+  }
+  return std::chrono::system_clock::from_time_t(utils::timeutils::mkgmtime(&end));
+}
+
+std::error_code processP12Certificate(const std::string& cert_file, const std::string& passphrase, const CertHandler& handler) {
+  utils::tls::BIO_unique_ptr fp{BIO_new(BIO_s_file())};
+  if (fp == nullptr) {
+    return get_last_ssl_error_code();
+  }
+  if (BIO_read_filename(fp.get(), cert_file.c_str()) <= 0) {
+    return get_last_ssl_error_code();
+  }
+  utils::tls::PKCS12_unique_ptr  p12{d2i_PKCS12_bio(fp.get(), nullptr)};
+  if (p12 == nullptr) {
+    return get_last_ssl_error_code();
+  }
+
+  EVP_PKEY* pkey = nullptr;
+  X509* cert = nullptr;
+  STACK_OF(X509)* ca = nullptr;
+  if (!PKCS12_parse(p12.get(), passphrase.c_str(), &pkey, &cert, &ca)) {
+    return get_last_ssl_error_code();
+  }
+  utils::tls::EVP_PKEY_unique_ptr pkey_ptr{pkey};
+  utils::tls::X509_unique_ptr cert_ptr{cert};
+  const auto ca_deleter = gsl::finally([ca] { sk_X509_pop_free(ca, X509_free); });
+
+  if (handler.cert_cb) {
+    if (auto error = handler.cert_cb(std::move(cert_ptr))) {
+      return error;
+    }
+  }
+
+  if (handler.chain_cert_cb) {
+    while (ca != nullptr && sk_X509_num(ca) > 0) {
+      if (auto error = handler.chain_cert_cb(utils::tls::X509_unique_ptr{sk_X509_pop(ca)})) {
+        return error;
+      }
+    }
+  }
+
+  if (handler.priv_key_cb) {
+    return handler.priv_key_cb(std::move(pkey_ptr));
+  }
+
+  return {};
+}
+
+std::error_code processPEMCertificate(const std::string& cert_file, const std::optional<std::string>& passphrase, const CertHandler& handler) {
+  utils::tls::BIO_unique_ptr fp{BIO_new(BIO_s_file())};
+  if (fp == nullptr) {
+    return get_last_ssl_error_code();
+  }
+  if (BIO_read_filename(fp.get(), cert_file.c_str()) <= 0) {
+    return get_last_ssl_error_code();
+  }
+  std::decay_t<decltype(pemPassWordCb)> pwd_cb = nullptr;
+  void* pwd_data = nullptr;
+  if (passphrase) {
+    pwd_cb = pemPassWordCb;
+    pwd_data = const_cast<std::string*>(&passphrase.value());
+  }
+
+  X509_unique_ptr cert{PEM_read_bio_X509_AUX(fp.get(), nullptr, pwd_cb, pwd_data)};
+  if (!cert) {
+    return get_last_ssl_error_code();
+  }
+
+  if (handler.cert_cb) {
+    if (auto error = handler.cert_cb(std::move(cert))) {
+      return error;
+    }
+  }
+
+  if (handler.chain_cert_cb) {
+    while (X509_unique_ptr chain_cert{PEM_read_bio_X509(fp.get(), nullptr, pwd_cb, pwd_data)}) {
+      if (auto error = handler.chain_cert_cb(std::move(chain_cert))) {
+        return error;
+      }
+    }
+  }
+
+  return {};
+}
+
 }  // namespace tls
 }  // namespace utils
 }  // namespace minifi