You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by GitBox <gi...@apache.org> on 2022/05/17 06:55:29 UTC

[GitHub] [nifi-minifi-cpp] adamdebreceni opened a new pull request, #1336: MINIFICPP-1826 - Warn on expiring certificate

adamdebreceni opened a new pull request, #1336:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1336

   Thank you for submitting a contribution to Apache NiFi - MiNiFi C++.
   
   In order to streamline the review of the contribution we ask you
   to ensure the following steps have been taken:
   
   ### For all changes:
   - [ ] Is there a JIRA ticket associated with this PR? Is it referenced
        in the commit message?
   
   - [ ] Does your PR title start with MINIFICPP-XXXX where XXXX is the JIRA number you are trying to resolve? Pay particular attention to the hyphen "-" character.
   
   - [ ] Has your PR been rebased against the latest commit within the target branch (typically main)?
   
   - [ ] Is your initial contribution a single, squashed commit?
   
   ### For code changes:
   - [ ] If adding new dependencies to the code, are these dependencies licensed in a way that is compatible for inclusion under [ASF 2.0](http://www.apache.org/legal/resolved.html#category-a)?
   - [ ] If applicable, have you updated the LICENSE file?
   - [ ] If applicable, have you updated the NOTICE file?
   
   ### For documentation related changes:
   - [ ] Have you ensured that format looks appropriate for the output in which it is rendered?
   
   ### Note:
   Please ensure that once the PR is submitted, you check GitHub Actions CI results for build issues and submit an update to your PR as soon as possible.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi-minifi-cpp] fgerlits commented on a diff in pull request #1336: MINIFICPP-1826 - Warn on expiring certificate

Posted by GitBox <gi...@apache.org>.
fgerlits commented on code in PR #1336:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1336#discussion_r875644534


##########
libminifi/src/utils/tls/CertificateUtils.cpp:
##########
@@ -33,6 +37,29 @@ namespace utils {
 namespace tls {
 
 #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());

Review Comment:
   it would be useful to log `GetLastError()` if this fails



##########
libminifi/src/utils/tls/CertificateUtils.cpp:
##########
@@ -116,6 +143,110 @@ EVP_PKEY_unique_ptr extractPrivateKey(const PCCERT_CONTEXT certificate) {
 }
 #endif  // WIN32
 
+std::string getLatestOpenSSLErrorString() {
+  unsigned long err = ERR_peek_last_error(); // NOLINT
+  if (err == 0U) {
+    return "";
+  }
+  char buf[4096];
+  ERR_error_string_n(err, buf, sizeof(buf));
+  return buf;
+}
+
+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(std::mktime(&end));

Review Comment:
   I think `ASN1_time_parse()` returns a UTC timestamp, but `mktime()` parses it as local time.  We have `utils::mkgmtime()` to parse a UTC timestamp.



##########
libminifi/src/controllers/SSLContextService.cpp:
##########
@@ -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::optional<std::string> {
+      if (SSL_CTX_use_certificate(ctx, cert.get()) != 1) {
+        return utils::StringUtils::join_pack("Failed to set certificate from ", certificate_, ", ", getLatestOpenSSLErrorString());

Review Comment:
   Most of these callbacks just log an error and return `nullopt`, but a few (like this one) return a string and the caller will log the error.  Why do we do this in two different ways?



##########
libminifi/include/utils/tls/CertificateUtils.h:
##########
@@ -43,14 +50,53 @@ 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);
+
+  bool isOpen() const;
+
+  PCCERT_CONTEXT nextCert();
+
+  ~WindowsCertStore();
+
+ private:
+  HCERTSTORE store_ptr_;
+  PCCERT_CONTEXT cert_ctx_ptr_ = nullptr;
+};
+
 // Returns nullptr on errors
 X509_unique_ptr convertWindowsCertificate(PCCERT_CONTEXT certificate);
 
 // Returns nullptr if the certificate has no associated private key, or the private key could not be extracted
 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::optional<std::string>(const X509_unique_ptr& cert)> cert_cb;
+  std::function<std::optional<std::string>(X509_unique_ptr cert)> chain_cert_cb;
+  std::function<std::optional<std::string>(const EVP_PKEY_unique_ptr& priv_key)> priv_key_cb;

Review Comment:
   Why is the signature of `chain_cert_cb` different?  And could all these functions take a `T*` or `const T&` instead of a `unique_ptr<T>`?



##########
libminifi/src/controllers/SSLContextService.cpp:
##########
@@ -563,6 +559,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::months{3}) {
+        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 '" << certificate_ << "'";

Review Comment:
   I think this should be
   ```suggestion
         core::logging::LOG_ERROR(logger_) << "Could not determine expiration date for certificate in '" << cert_file << "'";
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a diff in pull request #1336: MINIFICPP-1826 - Warn on expiring certificate

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on code in PR #1336:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1336#discussion_r876758518


##########
libminifi/src/controllers/SSLContextService.cpp:
##########
@@ -563,6 +559,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::months{3}) {
+        core::logging::LOG_WARN(logger_) << "Certificate in '" << cert_file << "' will expire at " << end_date_str;

Review Comment:
   changed it to 2 weeks



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi-minifi-cpp] szaszm commented on a diff in pull request #1336: MINIFICPP-1826 - Warn on expiring certificate

Posted by GitBox <gi...@apache.org>.
szaszm commented on code in PR #1336:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1336#discussion_r877083728


##########
libminifi/include/utils/tls/CertificateUtils.h:
##########
@@ -43,14 +50,53 @@ 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);
+
+  bool isOpen() const;
+
+  PCCERT_CONTEXT nextCert();
+
+  ~WindowsCertStore();
+
+ private:
+  HCERTSTORE store_ptr_;
+  PCCERT_CONTEXT cert_ctx_ptr_ = nullptr;
+};
+
 // Returns nullptr on errors
 X509_unique_ptr convertWindowsCertificate(PCCERT_CONTEXT certificate);
 
 // Returns nullptr if the certificate has no associated private key, or the private key could not be extracted
 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::optional<std::string>(const X509_unique_ptr& cert)> cert_cb;
+  std::function<std::optional<std::string>(X509_unique_ptr cert)> chain_cert_cb;
+  std::function<std::optional<std::string>(const EVP_PKEY_unique_ptr& priv_key)> priv_key_cb;
+};
+
+std::optional<std::string> processP12Certificate(const std::string& cert_file, const std::string& passphrase, const CertHandler& handler);
+
+std::optional<std::string> processPEMCertificate(const std::string& cert_file, const std::optional<std::string>& passphrase, const CertHandler& handler);

Review Comment:
   I think it's OK to depend on this, since there are platforms where `unsigned long` and `int` are the same size. I would try `gsl::narrow`, so that it would fail loudly if it ever doesn't fit.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi-minifi-cpp] szaszm commented on a diff in pull request #1336: MINIFICPP-1826 - Warn on expiring certificate

Posted by GitBox <gi...@apache.org>.
szaszm commented on code in PR #1336:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1336#discussion_r876200492


##########
libminifi/src/controllers/SSLContextService.cpp:
##########
@@ -563,6 +559,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::months{3}) {
+        core::logging::LOG_WARN(logger_) << "Certificate in '" << cert_file << "' will expire at " << end_date_str;

Review Comment:
   This makes sense, thanks Martin. Then only my first point remains, I think 2 weeks is enough notice. If someone is using Let's Encrypt, then they get 90 day certs, usually with automated renewal, which means they would get these warnings all the time.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a diff in pull request #1336: MINIFICPP-1826 - Warn on expiring certificate

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on code in PR #1336:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1336#discussion_r879244276


##########
libminifi/include/utils/tls/CertificateUtils.h:
##########
@@ -43,14 +50,53 @@ 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);
+
+  bool isOpen() const;
+
+  PCCERT_CONTEXT nextCert();
+
+  ~WindowsCertStore();
+
+ private:
+  HCERTSTORE store_ptr_;
+  PCCERT_CONTEXT cert_ctx_ptr_ = nullptr;
+};
+
 // Returns nullptr on errors
 X509_unique_ptr convertWindowsCertificate(PCCERT_CONTEXT certificate);
 
 // Returns nullptr if the certificate has no associated private key, or the private key could not be extracted
 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::optional<std::string>(const X509_unique_ptr& cert)> cert_cb;
+  std::function<std::optional<std::string>(X509_unique_ptr cert)> chain_cert_cb;
+  std::function<std::optional<std::string>(const EVP_PKEY_unique_ptr& priv_key)> priv_key_cb;
+};
+
+std::optional<std::string> processP12Certificate(const std::string& cert_file, const std::string& passphrase, const CertHandler& handler);
+
+std::optional<std::string> processPEMCertificate(const std::string& cert_file, const std::optional<std::string>& passphrase, const CertHandler& handler);

Review Comment:
   changed `std::optional<std::string>` to `std::error_code` with custom `ssl_error_category`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a diff in pull request #1336: MINIFICPP-1826 - Warn on expiring certificate

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on code in PR #1336:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1336#discussion_r876913104


##########
libminifi/src/controllers/SSLContextService.cpp:
##########
@@ -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::optional<std::string> {
+      if (SSL_CTX_use_certificate(ctx, cert.get()) != 1) {
+        return utils::StringUtils::join_pack("Failed to set certificate from ", certificate_, ", ", getLatestOpenSSLErrorString());

Review Comment:
   I think the only ones with "log-but-continue-on-error" behavior are the ones on `verifyCertificateExpiration` (even in this case "callback-unrelated" errors can occur and are logged)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a diff in pull request #1336: MINIFICPP-1826 - Warn on expiring certificate

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on code in PR #1336:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1336#discussion_r876746371


##########
libminifi/include/utils/tls/CertificateUtils.h:
##########
@@ -43,14 +50,53 @@ 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);
+
+  bool isOpen() const;
+
+  PCCERT_CONTEXT nextCert();
+
+  ~WindowsCertStore();
+
+ private:
+  HCERTSTORE store_ptr_;
+  PCCERT_CONTEXT cert_ctx_ptr_ = nullptr;
+};
+
 // Returns nullptr on errors
 X509_unique_ptr convertWindowsCertificate(PCCERT_CONTEXT certificate);
 
 // Returns nullptr if the certificate has no associated private key, or the private key could not be extracted
 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::optional<std::string>(const X509_unique_ptr& cert)> cert_cb;
+  std::function<std::optional<std::string>(X509_unique_ptr cert)> chain_cert_cb;
+  std::function<std::optional<std::string>(const EVP_PKEY_unique_ptr& priv_key)> priv_key_cb;

Review Comment:
   you are right, there is no point in passing `const unique_ptr<_>&`  to the other two, as for using `unique_ptr<_>`s, some functions (like `SSL_CTX_add_extra_chain_cert`) take ownership of the objects, so we would need to signal to the caller if such operations took place and make it the (callback) caller's responsibility to clean up, this way each callback assumes ownership and can locally decide if special logic needs to be applied



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a diff in pull request #1336: MINIFICPP-1826 - Warn on expiring certificate

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on code in PR #1336:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1336#discussion_r876876452


##########
libminifi/include/utils/tls/CertificateUtils.h:
##########
@@ -43,14 +50,53 @@ 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);
+
+  bool isOpen() const;
+
+  PCCERT_CONTEXT nextCert();
+
+  ~WindowsCertStore();
+
+ private:
+  HCERTSTORE store_ptr_;
+  PCCERT_CONTEXT cert_ctx_ptr_ = nullptr;
+};
+
 // Returns nullptr on errors
 X509_unique_ptr convertWindowsCertificate(PCCERT_CONTEXT certificate);
 
 // Returns nullptr if the certificate has no associated private key, or the private key could not be extracted
 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::optional<std::string>(const X509_unique_ptr& cert)> cert_cb;
+  std::function<std::optional<std::string>(X509_unique_ptr cert)> chain_cert_cb;
+  std::function<std::optional<std::string>(const EVP_PKEY_unique_ptr& priv_key)> priv_key_cb;
+};
+
+std::optional<std::string> processP12Certificate(const std::string& cert_file, const std::string& passphrase, const CertHandler& handler);
+
+std::optional<std::string> processPEMCertificate(const std::string& cert_file, const std::optional<std::string>& passphrase, const CertHandler& handler);

Review Comment:
   `ERR_peek_last_error` returns `unsigned long` which might be longer than `int` (for `std::error_code`), and although inspecting the implementation indicates that only the low 4 bytes are used, I'm not sure if we should depend on this



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi-minifi-cpp] martinzink commented on a diff in pull request #1336: MINIFICPP-1826 - Warn on expiring certificate

Posted by GitBox <gi...@apache.org>.
martinzink commented on code in PR #1336:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1336#discussion_r876165303


##########
libminifi/src/controllers/SSLContextService.cpp:
##########
@@ -563,6 +559,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::months{3}) {
+        core::logging::LOG_WARN(logger_) << "Certificate in '" << cert_file << "' will expire at " << end_date_str;

Review Comment:
   FYI std::chrono::months is different than std::chrono::month, first one is an exact duration (year length/12 (~30.4369 days)) the other one is the nth month of the year.
   e.g. timestamp(2022/03/01) - 1 months is in January not February
   If we want the same day of the previous month than we would need to use the calendar part of std::chrono/date.h https://en.cppreference.com/w/cpp/chrono/year_month_day/year_month_day (but we probably dont want that.)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi-minifi-cpp] martinzink commented on a diff in pull request #1336: MINIFICPP-1826 - Warn on expiring certificate

Posted by GitBox <gi...@apache.org>.
martinzink commented on code in PR #1336:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1336#discussion_r876165303


##########
libminifi/src/controllers/SSLContextService.cpp:
##########
@@ -563,6 +559,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::months{3}) {
+        core::logging::LOG_WARN(logger_) << "Certificate in '" << cert_file << "' will expire at " << end_date_str;

Review Comment:
   FYI std::chrono::months is different than std::chrono::month, first one is an exact duration (year length/12 (~30.4369 days)) the other one is the nth month of the year.
   e.g. timestamp(2022/03/01) - 1 month is in January not February
   If we want the same day of the previous month than we would need to use the calendar part of std::chrono/date.h https://en.cppreference.com/w/cpp/chrono/year_month_day/year_month_day (but we probably dont want that.)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi-minifi-cpp] szaszm commented on a diff in pull request #1336: MINIFICPP-1826 - Warn on expiring certificate

Posted by GitBox <gi...@apache.org>.
szaszm commented on code in PR #1336:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1336#discussion_r877083728


##########
libminifi/include/utils/tls/CertificateUtils.h:
##########
@@ -43,14 +50,53 @@ 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);
+
+  bool isOpen() const;
+
+  PCCERT_CONTEXT nextCert();
+
+  ~WindowsCertStore();
+
+ private:
+  HCERTSTORE store_ptr_;
+  PCCERT_CONTEXT cert_ctx_ptr_ = nullptr;
+};
+
 // Returns nullptr on errors
 X509_unique_ptr convertWindowsCertificate(PCCERT_CONTEXT certificate);
 
 // Returns nullptr if the certificate has no associated private key, or the private key could not be extracted
 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::optional<std::string>(const X509_unique_ptr& cert)> cert_cb;
+  std::function<std::optional<std::string>(X509_unique_ptr cert)> chain_cert_cb;
+  std::function<std::optional<std::string>(const EVP_PKEY_unique_ptr& priv_key)> priv_key_cb;
+};
+
+std::optional<std::string> processP12Certificate(const std::string& cert_file, const std::string& passphrase, const CertHandler& handler);
+
+std::optional<std::string> processPEMCertificate(const std::string& cert_file, const std::optional<std::string>& passphrase, const CertHandler& handler);

Review Comment:
   I think it's OK to depend on this, since there are platforms where `unsigned long` and `int` are the same size. I would try `gsl::narrow`, so that it would fail loudly if it ever doesn't fit.
   
   Asio does the same: https://github.com/chriskohlhoff/asio/blob/master/asio/include/asio/ssl/error.hpp#L102



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi-minifi-cpp] szaszm commented on a diff in pull request #1336: MINIFICPP-1826 - Warn on expiring certificate

Posted by GitBox <gi...@apache.org>.
szaszm commented on code in PR #1336:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1336#discussion_r876065172


##########
libminifi/src/controllers/SSLContextService.cpp:
##########
@@ -563,6 +559,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::months{3}) {
+        core::logging::LOG_WARN(logger_) << "Certificate in '" << cert_file << "' will expire at " << end_date_str;

Review Comment:
   I find this a bit excessive. WARN is enough for the last 2 (or maybe 4) weeks.
   
   Additionally, I would avoid comparing a precise duration to months, because months are variable length. I'm a bit surprised that this even compiles. (end_date - N months) < now() would be better, because then we know exactly which months to subtract and how long they are. If you change this to 2 weeks, which is plenty IMO, it will of course no longer be a problem.



##########
libminifi/include/utils/tls/CertificateUtils.h:
##########
@@ -43,14 +50,53 @@ 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);
+
+  bool isOpen() const;
+
+  PCCERT_CONTEXT nextCert();
+
+  ~WindowsCertStore();
+
+ private:
+  HCERTSTORE store_ptr_;
+  PCCERT_CONTEXT cert_ctx_ptr_ = nullptr;
+};
+
 // Returns nullptr on errors
 X509_unique_ptr convertWindowsCertificate(PCCERT_CONTEXT certificate);
 
 // Returns nullptr if the certificate has no associated private key, or the private key could not be extracted
 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::optional<std::string>(const X509_unique_ptr& cert)> cert_cb;
+  std::function<std::optional<std::string>(X509_unique_ptr cert)> chain_cert_cb;
+  std::function<std::optional<std::string>(const EVP_PKEY_unique_ptr& priv_key)> priv_key_cb;
+};
+
+std::optional<std::string> processP12Certificate(const std::string& cert_file, const std::string& passphrase, const CertHandler& handler);
+
+std::optional<std::string> processPEMCertificate(const std::string& cert_file, const std::optional<std::string>& passphrase, const CertHandler& handler);

Review Comment:
   I think a `std::error_code` return type with an internal openssl error category would better communicate the meaning and it would be similarly easy to use. See `DNS.cpp` for an example error_category definition.



##########
libminifi/src/utils/tls/CertificateUtils.cpp:
##########
@@ -33,6 +37,29 @@ namespace utils {
 namespace tls {
 
 #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());

Review Comment:
   About `GetLastError()`: The WINAPI way of converting it to string is complex and messy. This (copied from DNS.cpp) works for non-winsock errors as well, and it has a message member:
   ```
   std::error_code{WSAGetLastError(), std::system_category()}
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi-minifi-cpp] martinzink commented on a diff in pull request #1336: MINIFICPP-1826 - Warn on expiring certificate

Posted by GitBox <gi...@apache.org>.
martinzink commented on code in PR #1336:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1336#discussion_r876165303


##########
libminifi/src/controllers/SSLContextService.cpp:
##########
@@ -563,6 +559,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::months{3}) {
+        core::logging::LOG_WARN(logger_) << "Certificate in '" << cert_file << "' will expire at " << end_date_str;

Review Comment:
   FYI std::chrono::months is different than std::chrono::month, first one is an exact duration (year length/12 (~30.4369 days)) the other one is the nth month of the year. (same is true for std::chrono::days vs std::chrono::day)
   e.g. timestamp(2022/03/01) - 1 months is in January not February
   If we want the same day of the previous month than we would need to use the calendar part of std::chrono/date.h https://en.cppreference.com/w/cpp/chrono/year_month_day/year_month_day (but we probably dont want that.)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi-minifi-cpp] szaszm closed pull request #1336: MINIFICPP-1826 - Warn on expiring certificate

Posted by GitBox <gi...@apache.org>.
szaszm closed pull request #1336: MINIFICPP-1826 - Warn on expiring certificate
URL: https://github.com/apache/nifi-minifi-cpp/pull/1336


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a diff in pull request #1336: MINIFICPP-1826 - Warn on expiring certificate

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on code in PR #1336:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1336#discussion_r879243171


##########
libminifi/src/utils/tls/CertificateUtils.cpp:
##########
@@ -33,6 +37,29 @@ namespace utils {
 namespace tls {
 
 #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());

Review Comment:
   added error feedback



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a diff in pull request #1336: MINIFICPP-1826 - Warn on expiring certificate

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on code in PR #1336:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1336#discussion_r876908856


##########
libminifi/src/utils/tls/CertificateUtils.cpp:
##########
@@ -116,6 +143,110 @@ EVP_PKEY_unique_ptr extractPrivateKey(const PCCERT_CONTEXT certificate) {
 }
 #endif  // WIN32
 
+std::string getLatestOpenSSLErrorString() {
+  unsigned long err = ERR_peek_last_error(); // NOLINT
+  if (err == 0U) {
+    return "";
+  }
+  char buf[4096];
+  ERR_error_string_n(err, buf, sizeof(buf));
+  return buf;
+}
+
+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(std::mktime(&end));

Review Comment:
   changed it to `utils::mkgmtime`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi-minifi-cpp] martinzink commented on a diff in pull request #1336: MINIFICPP-1826 - Warn on expiring certificate

Posted by GitBox <gi...@apache.org>.
martinzink commented on code in PR #1336:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1336#discussion_r874964053


##########
libminifi/src/controllers/SSLContextService.cpp:
##########
@@ -563,6 +559,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());

Review Comment:
   its probably out of the scope of this PR, but I've added a jira https://issues.apache.org/jira/browse/MINIFICPP-1842 so getTimeStr accepts time_point



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a diff in pull request #1336: MINIFICPP-1826 - Warn on expiring certificate

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on code in PR #1336:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1336#discussion_r876758283


##########
libminifi/src/controllers/SSLContextService.cpp:
##########
@@ -563,6 +559,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::months{3}) {
+        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 '" << certificate_ << "'";

Review Comment:
   done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org