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

[nifi-minifi-cpp] branch master updated: MINIFICPP-965 Add other supported relationships to InvokeHTTP processor.

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

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


The following commit(s) were added to refs/heads/master by this push:
     new d63d338  MINIFICPP-965 Add other supported relationships to InvokeHTTP processor.
d63d338 is described below

commit d63d338dfffdb5009d953d5415a010c7cebcfd62
Author: Murtuza <ms...@gmail.com>
AuthorDate: Wed Apr 29 19:03:41 2020 -0400

    MINIFICPP-965 Add other supported relationships to InvokeHTTP processor.
    
    Added response, retry, no retry, failure relationships.
    Transfer request and response flow files appropriately.
    Overloaded virtual functions to set connection timeout and read timeout
    using std chrono
    Deprecated the old set timeout functions that use int64_t
    
    MINIFICPP-965 Add integration tests for invokehttp
    
    MINIFICPP-965 Add Invokehttp test with ssl support
    
    Signed-off-by: Arpad Boda <ab...@apache.org>
    
    This closes #775
---
 extensions/http-curl/client/HTTPClient.cpp         |  88 +++------
 extensions/http-curl/client/HTTPClient.h           |  55 ++++--
 extensions/http-curl/processors/InvokeHTTP.cpp     |  71 ++++----
 extensions/http-curl/processors/InvokeHTTP.h       |  30 +--
 extensions/http-curl/protocols/RESTSender.cpp      |   6 +-
 extensions/http-curl/sitetosite/HTTPProtocol.cpp   |   4 +-
 extensions/http-curl/tests/CMakeLists.txt          |   2 +
 extensions/http-curl/tests/HTTPHandlers.h          |  40 ++++
 .../http-curl/tests/VerifyInvokeHTTPTest.cpp       | 201 +++++++++++++++++++++
 extensions/mqtt/processors/ConvertUpdate.cpp       |   5 +-
 libminifi/include/core/Property.h                  |  12 ++
 libminifi/include/utils/HTTPClient.h               |  10 +-
 libminifi/src/RemoteProcessorGroupPort.cpp         |   4 +-
 libminifi/test/resources/C2VerifyServeResults.yml  |   4 +
 libminifi/test/resources/TestHTTPGet.yml           |   4 +
 libminifi/test/resources/TestHTTPGetSecure.yml     |   4 +
 libminifi/test/resources/TestHTTPPost.yml          |   6 +-
 .../test/resources/TestHTTPPostChunkedEncoding.yml |   4 +
 libminifi/test/resources/TestInvokeHTTPPost.yml    | 161 +++++++++++++++++
 .../test/resources/TestInvokeHTTPPostSecure.yml    | 175 ++++++++++++++++++
 20 files changed, 743 insertions(+), 143 deletions(-)

diff --git a/extensions/http-curl/client/HTTPClient.cpp b/extensions/http-curl/client/HTTPClient.cpp
index 354d62a..e35a21f 100644
--- a/extensions/http-curl/client/HTTPClient.cpp
+++ b/extensions/http-curl/client/HTTPClient.cpp
@@ -19,6 +19,7 @@
 #include "Exception.h"
 #include <memory>
 #include <climits>
+#include <cinttypes>
 #include <map>
 #include <vector>
 #include <string>
@@ -35,60 +36,17 @@ namespace utils {
 HTTPClient::HTTPClient(const std::string &url, const std::shared_ptr<minifi::controllers::SSLContextService> ssl_context_service)
     : core::Connectable("HTTPClient"),
       ssl_context_service_(ssl_context_service),
-      url_(url),
-      connect_timeout_(0),
-      read_timeout_(0),
-      content_type_str_(nullptr),
-      headers_(nullptr),
-      callback(nullptr),
-      write_callback_(nullptr),
-      http_code(0),
-      read_callback_(INT_MAX),
-      header_response_(-1),
-      res(CURLE_OK),
-      keep_alive_probe_(-1),
-      keep_alive_idle_(-1),
-      logger_(logging::LoggerFactory<HTTPClient>::getLogger()) {
+      url_(url) {
   http_session_ = curl_easy_init();
 }
 
 HTTPClient::HTTPClient(std::string name, utils::Identifier uuid)
-    : core::Connectable(name, uuid),
-      ssl_context_service_(nullptr),
-      url_(),
-      connect_timeout_(0),
-      read_timeout_(0),
-      content_type_str_(nullptr),
-      headers_(nullptr),
-      callback(nullptr),
-      write_callback_(nullptr),
-      http_code(0),
-      read_callback_(INT_MAX),
-      header_response_(-1),
-      res(CURLE_OK),
-      keep_alive_probe_(-1),
-      keep_alive_idle_(-1),
-      logger_(logging::LoggerFactory<HTTPClient>::getLogger()) {
+    : core::Connectable(name, uuid) {
   http_session_ = curl_easy_init();
 }
 
 HTTPClient::HTTPClient()
-    : core::Connectable("HTTPClient"),
-      ssl_context_service_(nullptr),
-      url_(),
-      connect_timeout_(0),
-      read_timeout_(0),
-      content_type_str_(nullptr),
-      headers_(nullptr),
-      callback(nullptr),
-      write_callback_(nullptr),
-      http_code(0),
-      read_callback_(INT_MAX),
-      header_response_(-1),
-      res(CURLE_OK),
-      keep_alive_probe_(-1),
-      keep_alive_idle_(-1),
-      logger_(logging::LoggerFactory<HTTPClient>::getLogger()) {
+    : core::Connectable("HTTPClient") {
   http_session_ = curl_easy_init();
 }
 
@@ -202,13 +160,20 @@ bool HTTPClient::setMinimumSSLVersion(SSLVersion minimum_version) {
   return ret == CURLE_OK;
 }
 
-void HTTPClient::setConnectionTimeout(int64_t timeout) {
-  connect_timeout_ = timeout;
-  curl_easy_setopt(http_session_, CURLOPT_NOSIGNAL, 1);
+DEPRECATED(/*deprecated in*/ 0.8.0, /*will remove in */ 2.0) void HTTPClient::setConnectionTimeout(int64_t timeout) {
+  setConnectionTimeout(std::chrono::milliseconds(timeout * 1000));
 }
 
-void HTTPClient::setReadTimeout(int64_t timeout) {
-  read_timeout_ = timeout;
+DEPRECATED(/*deprecated in*/ 0.8.0, /*will remove in */ 2.0) void HTTPClient::setReadTimeout(int64_t timeout) {
+  setReadTimeout(std::chrono::milliseconds(timeout * 1000));
+}
+
+void HTTPClient::setConnectionTimeout(std::chrono::milliseconds timeout) {
+  connect_timeout_ms_ = timeout;
+}
+
+void HTTPClient::setReadTimeout(std::chrono::milliseconds timeout) {
+  read_timeout_ms_ = timeout;
 }
 
 void HTTPClient::setReadCallback(HTTPReadCallback *callbackObj) {
@@ -278,10 +243,14 @@ void HTTPClient::setUseChunkedEncoding() {
 bool HTTPClient::submit() {
   if (IsNullOrEmpty(url_))
     return false;
-  if (connect_timeout_ > 0) {
-    curl_easy_setopt(http_session_, CURLOPT_CONNECTTIMEOUT, connect_timeout_);
+  curl_easy_setopt(http_session_, CURLOPT_NOSIGNAL, 1);
+  if (connect_timeout_ms_.count() > 0) {
+    curl_easy_setopt(http_session_, CURLOPT_CONNECTTIMEOUT_MS, connect_timeout_ms_.count());
   }
 
+  if (read_timeout_ms_.count() > 0) {
+    curl_easy_setopt(http_session_, CURLOPT_TIMEOUT_MS, read_timeout_ms_.count());
+  }
   if (headers_ != nullptr) {
     headers_ = curl_slist_append(headers_, "Expect:");
     curl_easy_setopt(http_session_, CURLOPT_HTTPHEADER, headers_);
@@ -296,12 +265,13 @@ bool HTTPClient::submit() {
   }
   curl_easy_setopt(http_session_, CURLOPT_HEADERFUNCTION, &utils::HTTPHeaderResponse::receive_headers);
   curl_easy_setopt(http_session_, CURLOPT_HEADERDATA, static_cast<void*>(&header_response_));
-  if (keep_alive_probe_ > 0){
-    logger_->log_debug("Setting keep alive to %d",keep_alive_probe_);
+  if (keep_alive_probe_.count() > 0) {
+    const auto keepAlive = std::chrono::duration_cast<std::chrono::seconds>(keep_alive_probe_);
+    const auto keepIdle = std::chrono::duration_cast<std::chrono::seconds>(keep_alive_idle_);
+    logger_->log_debug("Setting keep alive to %" PRId64 " seconds", keepAlive.count());
     curl_easy_setopt(http_session_, CURLOPT_TCP_KEEPALIVE, 1L);
-    curl_easy_setopt(http_session_, CURLOPT_TCP_KEEPINTVL, keep_alive_probe_);
-    curl_easy_setopt(http_session_, CURLOPT_TCP_KEEPIDLE, keep_alive_idle_);
-
+    curl_easy_setopt(http_session_, CURLOPT_TCP_KEEPINTVL, keepAlive.count());
+    curl_easy_setopt(http_session_, CURLOPT_TCP_KEEPIDLE, keepIdle.count());
   }
   else{
     logger_->log_debug("Not using keep alive");
@@ -314,7 +284,7 @@ bool HTTPClient::submit() {
   curl_easy_getinfo(http_session_, CURLINFO_RESPONSE_CODE, &http_code);
   curl_easy_getinfo(http_session_, CURLINFO_CONTENT_TYPE, &content_type_str_);
   if (res != CURLE_OK) {
-    logger_->log_error("curl_easy_perform() failed %s on %s\n", curl_easy_strerror(res), url_);
+    logger_->log_error("curl_easy_perform() failed %s on %s, error code %d\n", curl_easy_strerror(res), url_, res);
     return false;
   }
 
diff --git a/extensions/http-curl/client/HTTPClient.h b/extensions/http-curl/client/HTTPClient.h
index d00a7e9..e6e8fab 100644
--- a/extensions/http-curl/client/HTTPClient.h
+++ b/extensions/http-curl/client/HTTPClient.h
@@ -38,7 +38,7 @@
 #else
 #include <regex.h>
 #endif
-#include <vector>
+#include <chrono>
 
 #include "utils/ByteArrayCallback.h"
 #include "controllers/SSLContextService.h"
@@ -80,9 +80,19 @@ class HTTPClient : public BaseHTTPClient, public core::Connectable {
 
   virtual void initialize(const std::string &method, const std::string url = "", const std::shared_ptr<minifi::controllers::SSLContextService> ssl_context_service = nullptr) override;
 
-  virtual void setConnectionTimeout(int64_t timeout) override;
+  // This is a bad API and deprecated. Use the std::chrono variant of this
+  // It is assumed that the value of timeout provided to this function
+  // is in seconds units
+  DEPRECATED(/*deprecated in*/ 0.8.0, /*will remove in */ 2.0) virtual void setConnectionTimeout(int64_t timeout) override;
+
+  // This is a bad API and deprecated. Use the std::chrono variant of this
+  // It is assumed that the value of timeout provided to this function
+  // is in seconds units
+  DEPRECATED(/*deprecated in*/ 0.8.0, /*will remove in */ 2.0) virtual void setReadTimeout(int64_t timeout) override;
 
-  virtual void setReadTimeout(int64_t timeout) override;
+  virtual void setConnectionTimeout(std::chrono::milliseconds timeout) override;
+
+  virtual void setReadTimeout(std::chrono::milliseconds timeout) override;
 
   virtual void setUploadCallback(HTTPUploadCallback *callbackObj) override;
 
@@ -124,11 +134,19 @@ class HTTPClient : public BaseHTTPClient, public core::Connectable {
 
   bool setMinimumSSLVersion(SSLVersion minimum_version) override;
 
-  void setKeepAliveProbe(long probe){
+  DEPRECATED(/*deprecated in*/ 0.8.0, /*will remove in */ 2.0) void setKeepAliveProbe(long probe) {
+    keep_alive_probe_ = std::chrono::milliseconds(probe * 1000);
+  }
+
+  DEPRECATED(/*deprecated in*/ 0.8.0, /*will remove in */ 2.0) void setKeepAliveIdle(long idle) {
+    keep_alive_idle_ = std::chrono::milliseconds(idle * 1000);
+  }
+
+  void setKeepAliveProbe(std::chrono::milliseconds probe){
     keep_alive_probe_ = probe;
   }
 
-  void setKeepAliveIdle(long idle){
+  void setKeepAliveIdle(std::chrono::milliseconds idle){
     keep_alive_idle_= idle;
   }
 
@@ -229,30 +247,29 @@ class HTTPClient : public BaseHTTPClient, public core::Connectable {
 
   std::shared_ptr<minifi::controllers::SSLContextService> ssl_context_service_;
   std::string url_;
-  int64_t connect_timeout_;
+  std::chrono::milliseconds connect_timeout_ms_{0};
   // read timeout.
-  int64_t read_timeout_;
-  char *content_type_str_;
+  std::chrono::milliseconds read_timeout_ms_{0};
+  char *content_type_str_{nullptr};
   std::string content_type_;
-  struct curl_slist *headers_;
-  HTTPReadCallback *callback;
-  HTTPUploadCallback *write_callback_;
-  int64_t http_code;
-  ByteOutputCallback read_callback_;
-  utils::HTTPHeaderResponse header_response_;
+  struct curl_slist *headers_{nullptr};
+  HTTPReadCallback *callback{nullptr};
+  HTTPUploadCallback *write_callback_{nullptr};
+  int64_t http_code{0};
+  ByteOutputCallback read_callback_{INT_MAX};
+  utils::HTTPHeaderResponse header_response_{-1};
 
-  CURLcode res;
+  CURLcode res{CURLE_OK};
 
   CURL *http_session_;
 
   std::string method_;
 
-  long keep_alive_probe_;
-
-  long keep_alive_idle_;
+  std::chrono::milliseconds keep_alive_probe_{-1};
 
-  std::shared_ptr<logging::Logger> logger_;
+  std::chrono::milliseconds keep_alive_idle_{-1};
 
+  std::shared_ptr<logging::Logger> logger_{logging::LoggerFactory<HTTPClient>::getLogger()};
 };
 
 } /* namespace utils */
diff --git a/extensions/http-curl/processors/InvokeHTTP.cpp b/extensions/http-curl/processors/InvokeHTTP.cpp
index dbb28ad..de0464f 100644
--- a/extensions/http-curl/processors/InvokeHTTP.cpp
+++ b/extensions/http-curl/processors/InvokeHTTP.cpp
@@ -52,8 +52,6 @@ namespace nifi {
 namespace minifi {
 namespace processors {
 
-std::shared_ptr<utils::IdGenerator> InvokeHTTP::id_generator_ = utils::IdGenerator::getIdGenerator();
-
 const char *InvokeHTTP::ProcessorName = "InvokeHTTP";
 std::string InvokeHTTP::DefaultContentType = "application/octet-stream";
 
@@ -63,8 +61,14 @@ core::Property InvokeHTTP::Method("HTTP Method", "HTTP request method (GET, POST
 core::Property InvokeHTTP::URL(
     core::PropertyBuilder::createProperty("Remote URL")->withDescription("Remote URL which will be connected to, including scheme, host, port, path.")->isRequired(false)->supportsExpressionLanguage(
         true)->build());
-core::Property InvokeHTTP::ConnectTimeout("Connection Timeout", "Max wait time for connection to remote service.", "5 secs");
-core::Property InvokeHTTP::ReadTimeout("Read Timeout", "Max wait time for response from remote service.", "15 secs");
+
+core::Property InvokeHTTP::ConnectTimeout(
+      core::PropertyBuilder::createProperty("Connection Timeout")->withDescription("Max wait time for connection to remote service")->isRequired(false)
+         ->withDefaultValue<core::TimePeriodValue>("5 s")->build());
+
+core::Property InvokeHTTP::ReadTimeout(
+      core::PropertyBuilder::createProperty("Read Timeout")->withDescription("Max wait time for response from remote service")->isRequired(false)
+         ->withDefaultValue<core::TimePeriodValue>("15 s")->build());
 
 core::Property InvokeHTTP::DateHeader(
     core::PropertyBuilder::createProperty("Include Date Header")->withDescription("Include an RFC-2616 Date header in the request.")->isRequired(false)->withDefaultValue<bool>(true)->build());
@@ -149,9 +153,14 @@ void InvokeHTTP::initialize() {
 
   setSupportedProperties(properties);
   // Set the supported relationships
-  std::set<core::Relationship> relationships;
-  relationships.insert(Success);
-  setSupportedRelationships(relationships);
+  setSupportedRelationships({Success, RelResponse, RelFailure, RelRetry, RelNoRetry});
+}
+
+bool getTimeMSFromString(const std::string& propertyName, uint64_t& valInt) {
+  core::TimeUnit unit;
+  return !propertyName.empty()
+      && core::Property::StringToTime(propertyName, valInt, unit)
+      && core::Property::ConvertTimeUnitToMS(valInt, unit, valInt);
 }
 
 void InvokeHTTP::onSchedule(const std::shared_ptr<core::ProcessContext> &context, const std::shared_ptr<core::ProcessSessionFactory> &sessionFactory) {
@@ -165,15 +174,13 @@ void InvokeHTTP::onSchedule(const std::shared_ptr<core::ProcessContext> &context
     return;
   }
 
+  uint64_t valInt;
   std::string timeoutStr;
-
-  if (context->getProperty(ConnectTimeout.getName(), timeoutStr)) {
-    core::Property::StringToInt(timeoutStr, connect_timeout_);
-    // set the timeout in curl options.
-
+  if (context->getProperty(ConnectTimeout.getName(), timeoutStr)
+      && core::Property::getTimeMSFromString(timeoutStr, valInt)) {
+    connect_timeout_ms_ =  std::chrono::milliseconds(valInt);
   } else {
     logger_->log_debug("%s attribute is missing, so default value of %s will be used", ConnectTimeout.getName(), ConnectTimeout.getValue());
-
     return;
   }
 
@@ -182,9 +189,10 @@ void InvokeHTTP::onSchedule(const std::shared_ptr<core::ProcessContext> &context
     content_type_ = contentTypeStr;
   }
 
-  if (context->getProperty(ReadTimeout.getName(), timeoutStr)) {
-    core::Property::StringToInt(timeoutStr, read_timeout_);
-
+  timeoutStr.clear();
+  if (context->getProperty(ReadTimeout.getName(), timeoutStr)
+      && core::Property::getTimeMSFromString(timeoutStr, valInt)) {
+    read_timeout_ms_ =  std::chrono::milliseconds(valInt);
   } else {
     logger_->log_debug("%s attribute is missing, so default value of %s will be used", ReadTimeout.getName(), ReadTimeout.getValue());
   }
@@ -244,7 +252,7 @@ InvokeHTTP::~InvokeHTTP() {
 
 std::string InvokeHTTP::generateId() {
   utils::Identifier txId;
-  id_generator_->generate(txId);
+  utils::IdGenerator::getIdGenerator()->generate(txId);
   return txId.to_string();
 }
 
@@ -266,20 +274,19 @@ void InvokeHTTP::onTrigger(const std::shared_ptr<core::ProcessContext> &context,
       return;
     }
   } else {
-    context->getProperty(URL, url, flowFile);
     logger_->log_debug("InvokeHTTP -- Received flowfile");
   }
 
-  logger_->log_debug("onTrigger InvokeHTTP with %s to %s", method_, url);
+  logger_->log_debug("onTrigger InvokeHTTP with %s to %s", method_, url_);
 
   // create a transaction id
   std::string tx_id = generateId();
 
-  utils::HTTPClient client(url, ssl_context_service_);
+  utils::HTTPClient client(url_, ssl_context_service_);
 
   client.initialize(method_);
-  client.setConnectionTimeout(connect_timeout_);
-  client.setReadTimeout(read_timeout_);
+  client.setConnectionTimeout(connect_timeout_ms_);
+  client.setReadTimeout(read_timeout_ms_);
 
   if (!content_type_.empty()) {
     client.setContentType(content_type_);
@@ -333,9 +340,9 @@ void InvokeHTTP::onTrigger(const std::shared_ptr<core::ProcessContext> &context,
     int64_t http_code = client.getResponseCode();
     const char *content_type = client.getContentType();
     flowFile->addAttribute(STATUS_CODE, std::to_string(http_code));
-    if (response_headers.size() > 0)
+    if (!response_headers.empty())
       flowFile->addAttribute(STATUS_MESSAGE, response_headers.at(0));
-    flowFile->addAttribute(REQUEST_URL, url);
+    flowFile->addAttribute(REQUEST_URL, url_);
     flowFile->addAttribute(TRANSACTION_ID, tx_id);
 
     bool isSuccess = ((int32_t) (http_code / 100)) == 2;
@@ -355,18 +362,18 @@ void InvokeHTTP::onTrigger(const std::shared_ptr<core::ProcessContext> &context,
       // as per RFC 2046 -- 4.5.1
       response_flow->addKeyedAttribute(MIME_TYPE, content_type ? std::string(content_type) : DefaultContentType);
       response_flow->addAttribute(STATUS_CODE, std::to_string(http_code));
-      if (response_headers.size() > 0)
-        flowFile->addAttribute(STATUS_MESSAGE, response_headers.at(0));
+      if (!response_headers.empty())
+        response_flow->addAttribute(STATUS_MESSAGE, response_headers.at(0));
       response_flow->addAttribute(REQUEST_URL, url);
       response_flow->addAttribute(TRANSACTION_ID, tx_id);
       io::DataStream stream((const uint8_t*) response_body.data(), response_body.size());
       // need an import from the data stream.
       session->importFrom(stream, response_flow);
-    } else {
-      logger_->log_warn("Cannot output body to content");
-      response_flow = std::static_pointer_cast<FlowFileRecord>(session->create());
     }
     route(flowFile, response_flow, session, context, isSuccess, http_code);
+  } else {
+    session->penalize(flowFile);
+    session->transfer(flowFile, RelFailure);
   }
 }
 
@@ -381,7 +388,7 @@ void InvokeHTTP::route(std::shared_ptr<FlowFileRecord> &request, std::shared_ptr
   bool responseSent = false;
   if (always_output_response_ && response != nullptr) {
     logger_->log_debug("Outputting success and response");
-    session->transfer(response, Success);
+    session->transfer(response, RelResponse);
     responseSent = true;
   }
 
@@ -394,16 +401,14 @@ void InvokeHTTP::route(std::shared_ptr<FlowFileRecord> &request, std::shared_ptr
     }
     if (response != nullptr && !responseSent) {
       logger_->log_debug("Outputting success and response");
-      session->transfer(response, Success);
+      session->transfer(response, RelResponse);
     }
-
     // 5xx -> RETRY
   } else if (statusCode / 100 == 5) {
     if (request != nullptr) {
       session->penalize(request);
       session->transfer(request, RelRetry);
     }
-
     // 1xx, 3xx, 4xx -> NO RETRY
   } else {
     if (request != nullptr) {
diff --git a/extensions/http-curl/processors/InvokeHTTP.h b/extensions/http-curl/processors/InvokeHTTP.h
index 7c62aa2..15f0511 100644
--- a/extensions/http-curl/processors/InvokeHTTP.h
+++ b/extensions/http-curl/processors/InvokeHTTP.h
@@ -50,16 +50,7 @@ class InvokeHTTP : public core::Processor {
    * Create a new processor
    */
   InvokeHTTP(std::string name, utils::Identifier uuid = utils::Identifier())
-      : Processor(name, uuid),
-        ssl_context_service_(nullptr),
-        date_header_include_(true),
-        connect_timeout_(20000),
-        read_timeout_(20000),
-        always_output_response_(false),
-        use_chunked_encoding_(false),
-        penalize_no_retry_(false),
-        disable_peer_verification_(false),
-        logger_(logging::LoggerFactory<InvokeHTTP>::getLogger()) {
+      : Processor(name, uuid) {
   }
   // Destructor
   virtual ~InvokeHTTP();
@@ -140,35 +131,34 @@ class InvokeHTTP : public core::Processor {
    */
   bool emitFlowFile(const std::string &method);
 
-  std::shared_ptr<minifi::controllers::SSLContextService> ssl_context_service_;
+  std::shared_ptr<minifi::controllers::SSLContextService> ssl_context_service_{nullptr};
 
   // http method
   std::string method_;
   // url
   std::string url_;
   // include date in the header
-  bool date_header_include_;
+  bool date_header_include_{true};
   // attribute to send regex
   std::string attribute_to_send_regex_;
   // connection timeout
-  int64_t connect_timeout_;
+  std::chrono::milliseconds connect_timeout_ms_{20000};
   // read timeout.
-  int64_t read_timeout_;
+  std::chrono::milliseconds read_timeout_ms_{20000};
   // attribute in which response body will be added
   std::string put_attribute_name_;
   // determine if we always output a response.
-  bool always_output_response_;
+  bool always_output_response_{false};
   // content type.
   std::string content_type_;
   // use chunked encoding.
-  bool use_chunked_encoding_;
+  bool use_chunked_encoding_{false};
   // penalize on no retry
-  bool penalize_no_retry_;
+  bool penalize_no_retry_{false};
   // disable peer verification ( makes susceptible for MITM attacks )
-  bool disable_peer_verification_;
+  bool disable_peer_verification_{false};
  private:
-  std::shared_ptr<logging::Logger> logger_;
-  static std::shared_ptr<utils::IdGenerator> id_generator_;
+  std::shared_ptr<logging::Logger> logger_{logging::LoggerFactory<InvokeHTTP>::getLogger()};
 };
 
 REGISTER_RESOURCE(InvokeHTTP,"An HTTP client processor which can interact with a configurable HTTP Endpoint. "
diff --git a/extensions/http-curl/protocols/RESTSender.cpp b/extensions/http-curl/protocols/RESTSender.cpp
index fb46b54..160f048 100644
--- a/extensions/http-curl/protocols/RESTSender.cpp
+++ b/extensions/http-curl/protocols/RESTSender.cpp
@@ -104,9 +104,9 @@ const C2Payload RESTSender::sendPayload(const std::string url, const Direction d
 
   // Client declared last to make sure calbacks are still available when client is destructed
   utils::HTTPClient client(url, ssl_context_service_);
-  client.setKeepAliveProbe(2);
-  client.setKeepAliveIdle(2);
-  client.setConnectionTimeout(2);
+  client.setKeepAliveProbe(std::chrono::milliseconds(2000));
+  client.setKeepAliveIdle(std::chrono::milliseconds(2000));
+  client.setConnectionTimeout(std::chrono::milliseconds(2000));
   if (direction == Direction::TRANSMIT) {
     input = std::unique_ptr<utils::ByteInputCallBack>(new utils::ByteInputCallBack());
     callback = std::unique_ptr<utils::HTTPUploadCallback>(new utils::HTTPUploadCallback());
diff --git a/extensions/http-curl/sitetosite/HTTPProtocol.cpp b/extensions/http-curl/sitetosite/HTTPProtocol.cpp
index abf8bdd..bb60011 100644
--- a/extensions/http-curl/sitetosite/HTTPProtocol.cpp
+++ b/extensions/http-curl/sitetosite/HTTPProtocol.cpp
@@ -55,7 +55,7 @@ std::shared_ptr<Transaction> HttpSiteToSiteClient::createTransaction(std::string
   uri << getBaseURI() << "data-transfer/" << dir_str << "/" << getPortId() << "/transactions";
   auto client = create_http_client(uri.str(), "POST");
   client->appendHeader(PROTOCOL_VERSION_HEADER, "1");
-  client->setConnectionTimeout(5);
+  client->setConnectionTimeout(std::chrono::milliseconds(5000));
   client->setContentType("application/json");
   client->appendHeader("Accept: application/json");
   client->setUseChunkedEncoding();
@@ -282,7 +282,7 @@ void HttpSiteToSiteClient::closeTransaction(const std::string &transactionID) {
 
   client->appendHeader(PROTOCOL_VERSION_HEADER, "1");
 
-  client->setConnectionTimeout(5);
+  client->setConnectionTimeout(std::chrono::milliseconds(5000));
 
   client->appendHeader("Accept", "application/json");
 
diff --git a/extensions/http-curl/tests/CMakeLists.txt b/extensions/http-curl/tests/CMakeLists.txt
index f01c4f9..d619a75 100644
--- a/extensions/http-curl/tests/CMakeLists.txt
+++ b/extensions/http-curl/tests/CMakeLists.txt
@@ -80,6 +80,7 @@ add_test(NAME C2NullConfiguration COMMAND C2NullConfiguration "${TEST_RESOURCES}
 if(NOT OPENSSL_OFF)
 add_test(NAME HttpGetIntegrationTestSecure COMMAND HttpGetIntegrationTest "${TEST_RESOURCES}/TestHTTPGetSecure.yml"  "${TEST_RESOURCES}/")
 add_test(NAME C2VerifyHeartbeatAndStopSecure COMMAND C2VerifyHeartbeatAndStop "${TEST_RESOURCES}/C2VerifyHeartbeatAndStopSecure.yml" "${TEST_RESOURCES}/")
+add_test(NAME VerifyInvokeHTTPTestSecure COMMAND VerifyInvokeHTTPTest "${TEST_RESOURCES}/TestInvokeHTTPPostSecure.yml" "${TEST_RESOURCES}/")
 endif()
 add_test(NAME HttpPostIntegrationTest COMMAND HttpPostIntegrationTest "${TEST_RESOURCES}/TestHTTPPost.yml" "${TEST_RESOURCES}/")
 if (NOT APPLE)
@@ -91,3 +92,4 @@ add_test(NAME HTTPSiteToSiteTests COMMAND HTTPSiteToSiteTests "${TEST_RESOURCES}
 add_test(NAME SiteToSiteRestTest COMMAND SiteToSiteRestTest "${TEST_RESOURCES}/TestSite2SiteRest.yml" "${TEST_RESOURCES}/" "http://localhost:8077/nifi-api/site-to-site")
 add_test(NAME ControllerServiceIntegrationTests COMMAND ControllerServiceIntegrationTests "${TEST_RESOURCES}/TestControllerServices.yml" "${TEST_RESOURCES}/")
 add_test(NAME ThreadPoolAdjust COMMAND ThreadPoolAdjust "${TEST_RESOURCES}/ThreadPoolAdjust.yml" "${TEST_RESOURCES}/")
+add_test(NAME VerifyInvokeHTTPTest COMMAND VerifyInvokeHTTPTest "${TEST_RESOURCES}/TestInvokeHTTPPost.yml")
diff --git a/extensions/http-curl/tests/HTTPHandlers.h b/extensions/http-curl/tests/HTTPHandlers.h
index 8da7b34..5abbc60 100644
--- a/extensions/http-curl/tests/HTTPHandlers.h
+++ b/extensions/http-curl/tests/HTTPHandlers.h
@@ -444,4 +444,44 @@ class HeartbeatHandler : public CivetHandler {
   bool isSecure;
 };
 
+class InvokeHTTPCouldNotConnectHandler : public CivetHandler {
+};
+
+class InvokeHTTPResponseOKHandler : public CivetHandler {
+public:
+  bool handlePost(CivetServer *, struct mg_connection *conn) {
+    mg_printf(conn, "HTTP/1.1 201 OK\r\nContent-Type: text/plain\r\nContent-Length: 0\r\nConnection: close\r\n\r\n");
+    return true;
+  }
+};
+
+class InvokeHTTPResponse404Handler : public CivetHandler {
+public:
+  bool handlePost(CivetServer *, struct mg_connection *conn) {
+    mg_printf(conn, "HTTP/1.1 404 Not Found\r\nContent-Type: text/plain\r\nContent-Length: 0\r\nConnection: close\r\n\r\n");
+    return true;
+  }
+};
+
+class InvokeHTTPResponse501Handler : public CivetHandler {
+public:
+  bool handlePost(CivetServer *, struct mg_connection *conn) {
+    mg_printf(conn, "HTTP/1.1 501 Not Implemented\r\nContent-Type: text/plain\r\nContent-Length: 0\r\nConnection: close\r\n\r\n");
+    return true;
+  }
+};
+
+class InvokeHTTPResponseTimeoutHandler : public CivetHandler {
+public:
+    InvokeHTTPResponseTimeoutHandler(std::chrono::milliseconds wait_ms)
+        : wait_(wait_ms) {
+    }
+  bool handlePost(CivetServer *, struct mg_connection *conn) {
+    std::this_thread::sleep_for(wait_);
+    mg_printf(conn, "HTTP/1.1 201 OK\r\nContent-Type: text/plain\r\nContent-Length: 0\r\nConnection: close\r\n\r\n");
+    return true;
+  }
+protected:
+  std::chrono::milliseconds wait_;
+};
 #endif /* LIBMINIFI_TEST_CURL_TESTS_SITETOSITEHTTP_HTTPHANDLERS_H_ */
diff --git a/extensions/http-curl/tests/VerifyInvokeHTTPTest.cpp b/extensions/http-curl/tests/VerifyInvokeHTTPTest.cpp
new file mode 100644
index 0000000..66c8525
--- /dev/null
+++ b/extensions/http-curl/tests/VerifyInvokeHTTPTest.cpp
@@ -0,0 +1,201 @@
+/**
+ *
+ * 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.
+ */
+
+#undef NDEBUG
+#include "TestBase.h"
+#include "HTTPHandlers.h"
+#include "HTTPClient.h"
+#include "InvokeHTTP.h"
+#include "processors/LogAttribute.h"
+#include "core/state/ProcessorController.h"
+
+#include "../tests/TestServer.h"
+#include "CivetServer.h"
+#include "HTTPIntegrationBase.h"
+
+class VerifyInvokeHTTP : public CoapIntegrationBase {
+public:
+  VerifyInvokeHTTP()
+      : CoapIntegrationBase(6000) {
+  }
+
+  virtual void testSetup() override {
+    LogTestController::getInstance().setDebug<utils::HTTPClient>();
+    LogTestController::getInstance().setDebug<LogTestController>();
+    LogTestController::getInstance().setTrace<minifi::processors::InvokeHTTP>();
+    LogTestController::getInstance().setTrace<minifi::processors::LogAttribute>();
+  }
+
+  virtual void cleanup() override {
+  }
+
+  void setProperties(std::shared_ptr<core::Processor> proc) {
+    std::string url = scheme + "://localhost:" + getWebPort() + path;
+    proc->setProperty(minifi::processors::InvokeHTTP::URL.getName(), url);
+  }
+
+  void setupFlow(const std::string& flow_yml_path) {
+    testSetup();
+
+    std::shared_ptr<core::Repository> test_repo = std::make_shared<TestRepository>();
+    std::shared_ptr<core::Repository> test_flow_repo = std::make_shared<TestFlowRepository>();
+
+    configuration->set(minifi::Configure::nifi_flow_configuration_file, flow_yml_path);
+
+    std::shared_ptr<core::ContentRepository> content_repo = std::make_shared<core::repository::VolatileContentRepository>();
+    content_repo->initialize(configuration);
+    std::shared_ptr<minifi::io::StreamFactory> stream_factory = minifi::io::StreamFactory::getInstance(configuration);
+    std::unique_ptr<core::FlowConfiguration> yaml_ptr = std::unique_ptr<core::YamlConfiguration>(
+        new core::YamlConfiguration(test_repo, test_repo, content_repo, stream_factory, configuration, flow_yml_path));
+
+    flowController_ = std::make_shared<minifi::FlowController>(test_repo, test_flow_repo, configuration, std::move(yaml_ptr), content_repo, DEFAULT_ROOT_GROUP_NAME, true);
+    flowController_->load();
+
+    const auto components = flowController_->getComponents("InvokeHTTP");
+    assert(!components.empty());
+
+    const auto stateController = components.at(0);
+    assert(stateController);
+    const auto processorController = std::dynamic_pointer_cast<minifi::state::ProcessorController>(stateController);
+    assert(processorController);
+    setProperties(processorController->getProcessor());
+  }
+
+  virtual void run(std::string flow_yml_path) override {
+    setupFlow(flow_yml_path);
+    startFlowController();
+
+    waitToVerifyProcessor();
+    shutdownBeforeFlowController();
+    stopFlowController();
+  }
+
+  void startFlowController() {
+    flowController_->start();
+  }
+
+  void stopFlowController() {
+    flowController_->unload();
+    flowController_->stopC2();
+
+    runAssertions();
+    cleanup();
+  }
+};
+
+class VerifyInvokeHTTPOKResponse : public VerifyInvokeHTTP {
+public:
+  virtual void runAssertions() override {
+    assert(LogTestController::getInstance().contains("key:invokehttp.status.code value:201"));
+    assert(LogTestController::getInstance().contains("response code 201"));
+  }
+};
+
+class VerifyCouldNotConnectInvokeHTTP : public VerifyInvokeHTTP {
+public:
+  virtual void runAssertions() override {
+    assert(LogTestController::getInstance().contains("key:invoke_http value:failure"));
+  }
+};
+
+class VerifyNoRetryInvokeHTTP : public VerifyInvokeHTTP {
+public:
+  virtual void runAssertions() override {
+    assert(LogTestController::getInstance().contains("key:invokehttp.status.message value:HTTP/1.1 404 Not Found"));
+    assert(LogTestController::getInstance().contains("isSuccess: 0, response code 404"));
+  }
+};
+
+class VerifyRetryInvokeHTTP : public VerifyInvokeHTTP {
+public:
+  virtual void runAssertions() override {
+    assert(LogTestController::getInstance().contains("key:invokehttp.status.message value:HTTP/1.1 501 Not Implemented"));
+    assert(LogTestController::getInstance().contains("isSuccess: 0, response code 501"));
+  }
+};
+
+class VerifyRWTimeoutInvokeHTTP : public VerifyInvokeHTTP {
+public:
+  virtual void runAssertions() override {
+    assert(LogTestController::getInstance().contains("key:invoke_http value:failure"));
+    assert(LogTestController::getInstance().contains("failed Timeout was reached"));
+  }
+};
+
+void run(VerifyInvokeHTTP& harness,
+    const std::string& url,
+    const std::string& test_file_location,
+    const std::string& key_dir,
+    CivetHandler * handler) {
+
+  harness.setKeyDir(key_dir);
+  harness.setUrl(url, handler);
+  harness.run(test_file_location);
+}
+
+int main(int argc, char ** argv) {
+  std::string key_dir, test_file_location, url;
+  if (argc > 1) {
+    test_file_location = argv[1];
+    url = "http://localhost:0/minifi";
+    if (argc > 2) {
+      key_dir = argv[2];
+      url = "https://localhost:0/minifi";
+    }
+  }
+
+  // Stop civet server to simulate
+  // unreachable remote end point
+  {
+    InvokeHTTPCouldNotConnectHandler handler;
+    VerifyCouldNotConnectInvokeHTTP harness;
+    harness.setKeyDir(key_dir);
+    harness.setUrl(url, &handler);
+    harness.setupFlow(test_file_location);
+    harness.shutdownBeforeFlowController();
+    harness.startFlowController();
+    harness.waitToVerifyProcessor();
+    harness.stopFlowController();
+  }
+
+  {
+    InvokeHTTPResponseOKHandler handler;
+    VerifyInvokeHTTPOKResponse harness;
+    run(harness, url, test_file_location, key_dir, &handler);
+  }
+
+  {
+    InvokeHTTPResponse404Handler handler;
+    VerifyNoRetryInvokeHTTP harness;
+    run(harness, url, test_file_location, key_dir, &handler);
+  }
+
+  {
+    InvokeHTTPResponse501Handler handler;
+    VerifyRetryInvokeHTTP harness;
+    run(harness, url, test_file_location, key_dir, &handler);
+  }
+
+  {
+    InvokeHTTPResponseTimeoutHandler handler(std::chrono::milliseconds(4000));
+    VerifyRWTimeoutInvokeHTTP harness;
+    run(harness, url, test_file_location, key_dir, &handler);
+  }
+
+  return 0;
+}
diff --git a/extensions/mqtt/processors/ConvertUpdate.cpp b/extensions/mqtt/processors/ConvertUpdate.cpp
index 264a4e0..b0689b0 100644
--- a/extensions/mqtt/processors/ConvertUpdate.cpp
+++ b/extensions/mqtt/processors/ConvertUpdate.cpp
@@ -19,6 +19,7 @@
 #include "utils/HTTPClient.h"
 #include "io/BaseStream.h"
 #include "io/DataStream.h"
+
 namespace org {
 namespace apache {
 namespace nifi {
@@ -62,8 +63,8 @@ void ConvertUpdate::onTrigger(const std::shared_ptr<core::ProcessContext> &conte
       }
       std::unique_ptr<utils::BaseHTTPClient> client = std::unique_ptr<utils::BaseHTTPClient>(dynamic_cast<utils::BaseHTTPClient*>(client_ptr));
       client->initialize("GET");
-      client->setConnectionTimeout(2000);
-      client->setReadTimeout(2000);
+      client->setConnectionTimeout(std::chrono::milliseconds(2000));
+      client->setReadTimeout(std::chrono::milliseconds(2000));
 
       if (client->submit()) {
         auto data = client->getResponseBody();
diff --git a/libminifi/include/core/Property.h b/libminifi/include/core/Property.h
index 8921e59..a374998 100644
--- a/libminifi/include/core/Property.h
+++ b/libminifi/include/core/Property.h
@@ -412,6 +412,18 @@ class Property {
     return true;
   }
 
+  static bool getTimeMSFromString(const std::string& str, uint64_t& valInt) {
+    core::TimeUnit unit;
+    return !str.empty() && StringToTime(str, valInt, unit)
+        && ConvertTimeUnitToMS(valInt, unit, valInt);
+  }
+
+  static bool getTimeMSFromString(const std::string& str, int64_t& valInt) {
+    core::TimeUnit unit;
+    return !str.empty() && StringToTime(str, valInt, unit)
+        && ConvertTimeUnitToMS(valInt, unit, valInt);
+  }
+
   // Convert String to Integer
   template<typename T>
   static bool StringToInt(std::string input, T &output) {
diff --git a/libminifi/include/utils/HTTPClient.h b/libminifi/include/utils/HTTPClient.h
index 46a066d..2998115 100644
--- a/libminifi/include/utils/HTTPClient.h
+++ b/libminifi/include/utils/HTTPClient.h
@@ -283,10 +283,16 @@ public:
   virtual void initialize(const std::string &method, const std::string url = "", const std::shared_ptr<minifi::controllers::SSLContextService> ssl_context_service = nullptr) {
   }
 
-  virtual void setConnectionTimeout(int64_t timeout) {
+  DEPRECATED(/*deprecated in*/ 0.8.0, /*will remove in */ 2.0) virtual void setConnectionTimeout(int64_t timeout) {
   }
 
-  virtual void setReadTimeout(int64_t timeout) {
+  DEPRECATED(/*deprecated in*/ 0.8.0, /*will remove in */ 2.0) virtual void setReadTimeout(int64_t timeout) {
+  }
+
+  virtual void setConnectionTimeout(std::chrono::milliseconds timeout) {
+  }
+
+  virtual void setReadTimeout(std::chrono::milliseconds timeout) {
   }
 
   virtual void setUploadCallback(HTTPUploadCallback *callbackObj) {
diff --git a/libminifi/src/RemoteProcessorGroupPort.cpp b/libminifi/src/RemoteProcessorGroupPort.cpp
index 78ab716..f110e89 100644
--- a/libminifi/src/RemoteProcessorGroupPort.cpp
+++ b/libminifi/src/RemoteProcessorGroupPort.cpp
@@ -305,7 +305,7 @@ std::pair<std::string, int> RemoteProcessorGroupPort::refreshRemoteSite2SiteInfo
       client->initialize("GET", loginUrl.str(), ssl_service);
       // use a connection timeout. if this times out we will simply attempt re-connection
       // so no need for configuration parameter that isn't already defined in Processor
-      client->setConnectionTimeout(10);
+      client->setConnectionTimeout(std::chrono::milliseconds(10000));
 
       token = utils::get_token(client.get(), this->rest_user_name_, this->rest_password_);
       logger_->log_debug("Token from NiFi REST Api endpoint %s,  %s", loginUrl.str(), token);
@@ -323,7 +323,7 @@ std::pair<std::string, int> RemoteProcessorGroupPort::refreshRemoteSite2SiteInfo
     client->initialize("GET", fullUrl.str().c_str(), ssl_service);
     // use a connection timeout. if this times out we will simply attempt re-connection
     // so no need for configuration parameter that isn't already defined in Processor
-    client->setConnectionTimeout(10);
+    client->setConnectionTimeout(std::chrono::milliseconds(10000));
     if (!proxy_.host.empty()) {
       client->setHTTPProxy(proxy_);
     }
diff --git a/libminifi/test/resources/C2VerifyServeResults.yml b/libminifi/test/resources/C2VerifyServeResults.yml
index 18db3dd..cb432d1 100644
--- a/libminifi/test/resources/C2VerifyServeResults.yml
+++ b/libminifi/test/resources/C2VerifyServeResults.yml
@@ -30,6 +30,10 @@ Processors:
       yield period: 1 sec
       run duration nanos: 0
       auto-terminated relationships list:
+          - failure
+          - retry
+          - no retry
+          - response
       Properties:
           HTTP Method: GET
           Remote URL: http://localhost:10013/geturl
diff --git a/libminifi/test/resources/TestHTTPGet.yml b/libminifi/test/resources/TestHTTPGet.yml
index 83de7ac..fc4a07f 100644
--- a/libminifi/test/resources/TestHTTPGet.yml
+++ b/libminifi/test/resources/TestHTTPGet.yml
@@ -30,6 +30,10 @@ Processors:
       yield period: 1 sec
       run duration nanos: 0
       auto-terminated relationships list:
+          - retry
+          - no retry
+          - response
+          - failure
       Properties:
           HTTP Method: GET
           Remote URL: http://localhost:10003/geturl
diff --git a/libminifi/test/resources/TestHTTPGetSecure.yml b/libminifi/test/resources/TestHTTPGetSecure.yml
index a5a0bee..d90770d 100644
--- a/libminifi/test/resources/TestHTTPGetSecure.yml
+++ b/libminifi/test/resources/TestHTTPGetSecure.yml
@@ -30,6 +30,10 @@ Processors:
       yield period: 1 sec
       run duration nanos: 0
       auto-terminated relationships list:
+          - failure
+          - retry
+          - no retry
+          - response
       Properties:
           SSL Context Service: SSLContextService
           HTTP Method: GET
diff --git a/libminifi/test/resources/TestHTTPPost.yml b/libminifi/test/resources/TestHTTPPost.yml
index 3f01fa8..10248f7 100644
--- a/libminifi/test/resources/TestHTTPPost.yml
+++ b/libminifi/test/resources/TestHTTPPost.yml
@@ -53,7 +53,11 @@ Processors:
       penalization period: 30 sec
       yield period: 1 sec
       run duration nanos: 0
-      auto-terminated relationships list: 
+      auto-terminated relationships list:
+         - retry
+         - no retry
+         - failure
+         - response
       Properties:
           HTTP Method: POST
           Content-type: text/html
diff --git a/libminifi/test/resources/TestHTTPPostChunkedEncoding.yml b/libminifi/test/resources/TestHTTPPostChunkedEncoding.yml
index f31f3e7..993a289 100644
--- a/libminifi/test/resources/TestHTTPPostChunkedEncoding.yml
+++ b/libminifi/test/resources/TestHTTPPostChunkedEncoding.yml
@@ -55,6 +55,10 @@ Processors:
       run duration nanos: 0
       auto-terminated relationships list: 
           - success
+          - retry
+          - failure
+          - response
+          - no retry
       Properties:
           HTTP Method: POST
           Use Chunked Encoding: true
diff --git a/libminifi/test/resources/TestInvokeHTTPPost.yml b/libminifi/test/resources/TestInvokeHTTPPost.yml
new file mode 100644
index 0000000..9450edf
--- /dev/null
+++ b/libminifi/test/resources/TestInvokeHTTPPost.yml
@@ -0,0 +1,161 @@
+MiNiFi Config Version: 3
+Flow Controller:
+  name: c++lw
+  comment: Created by MiNiFi C2 Flow Designer
+Core Properties:
+  flow controller graceful shutdown period: 10 sec
+  flow service write delay interval: 500 ms
+  administrative yield duration: 30 sec
+  bored yield duration: 10 millis
+  max concurrent threads: 1
+  variable registry properties: ''
+FlowFile Repository:
+  partitions: 256
+  checkpoint interval: 2 mins
+  always sync: false
+  Swap:
+    threshold: 20000
+    in period: 5 sec
+    in threads: 1
+    out period: 5 sec
+    out threads: 4
+Content Repository:
+  content claim max appendable size: 10 MB
+  content claim max flow files: 100
+  always sync: false
+Provenance Repository:
+  provenance rollover time: 1 min
+  implementation: org.apache.nifi.provenance.MiNiFiPersistentProvenanceRepository
+Component Status Repository:
+  buffer size: 1440
+  snapshot frequency: 1 min
+Security Properties:
+  keystore: ''
+  keystore type: ''
+  keystore password: ''
+  key password: ''
+  truststore: ''
+  truststore type: ''
+  truststore password: ''
+  ssl protocol: ''
+  Sensitive Props:
+    key:
+    algorithm: PBEWITHMD5AND256BITAES-CBC-OPENSSL
+    provider: BC
+Processors:
+- id: 4ed2d51d-076a-49b0-88de-5cf5adf52a7e
+  name: GenerateFlowFile
+  class: org.apache.nifi.minifi.processors.GenerateFlowFile
+  max concurrent tasks: 1
+  scheduling strategy: TIMER_DRIVEN
+  scheduling period: 15000 ms
+  penalization period: 1000 ms
+  yield period: 1000 ms
+  run duration nanos: 0
+  auto-terminated relationships list: []
+  Properties:
+    Batch Size: '1'
+    Data Format: Binary
+    File Size: 1 kB
+    Unique FlowFiles: 'true'
+- id: 1d51724d-dd76-46a0-892d-a7c7408d58dd
+  name: InvokeHTTP
+  class: org.apache.nifi.minifi.processors.InvokeHTTP
+  max concurrent tasks: 1
+  scheduling strategy: TIMER_DRIVEN
+  scheduling period: 1000 ms
+  penalization period: 1000 ms
+  yield period: 1000 ms
+  run duration nanos: 0
+  auto-terminated relationships list: []
+  Properties:
+    Always Output Response: 'false'
+    Connection Timeout: 3 s
+    Content-type: application/octet-stream
+    Disable Peer Verification: 'false'
+    HTTP Method: POST
+    Include Date Header: 'true'
+    Read Timeout: 4 s
+    Remote URL: http://localhost:0/minifi
+    Use Chunked Encoding: 'false'
+    send-message-body: 'true'
+- id: 8543b2e2-a373-4d8d-8710-ac314666696b
+  name: LogAttribute
+  class: org.apache.nifi.minifi.processors.LogAttribute
+  max concurrent tasks: 1
+  scheduling strategy: TIMER_DRIVEN
+  scheduling period: 1000 ms
+  penalization period: 30000 ms
+  yield period: 1000 ms
+  run duration nanos: 0
+  auto-terminated relationships list:
+  - success
+  Properties:
+    FlowFiles To Log: '1'
+    Hexencode Payload: 'false'
+    Log Payload: 'false'
+    Maximum Payload Line Length: '80'
+- id: 4f8a0c9b-6f7c-458f-9b4d-480addb2e0af
+  name: UpdateAttribute
+  class: org.apache.nifi.minifi.processors.UpdateAttribute
+  max concurrent tasks: 1
+  scheduling strategy: TIMER_DRIVEN
+  scheduling period: 200 ms
+  penalization period: 1000 ms
+  yield period: 1000 ms
+  run duration nanos: 0
+  auto-terminated relationships list:
+  - failure
+  Properties:
+    invoke_http: failure
+Controller Services: []
+Process Groups: []
+Input Ports: []
+Output Ports: []
+Funnels: []
+Connections:
+- id: 189bee81-3e56-45ac-b6bb-aa70b7180dc8
+  name: GenerateFlowFile/success/InvokeHTTP
+  source id: 4ed2d51d-076a-49b0-88de-5cf5adf52a7e
+  source relationship names:
+  - success
+  destination id: 1d51724d-dd76-46a0-892d-a7c7408d58dd
+  max work queue size: 0
+  max work queue data size: 10000 B
+  flowfile expiration: 0 seconds
+  queue prioritizer class: ''
+- id: f5d1dd2d-f675-41bd-bf1b-b571aa23ebfb
+  name: InvokeHTTP/failure/UpdateAttribute
+  source id: 1d51724d-dd76-46a0-892d-a7c7408d58dd
+  source relationship names:
+  - failure
+  destination id: 4f8a0c9b-6f7c-458f-9b4d-480addb2e0af
+  max work queue size: 0
+  max work queue data size: 10000 B
+  flowfile expiration: 0 seconds
+  queue prioritizer class: ''
+- id: 000035d8-46a0-460a-809c-f0c6320d532e
+  name: InvokeHTTP/response/LogAttribute
+  source id: 1d51724d-dd76-46a0-892d-a7c7408d58dd
+  source relationship names:
+  - no retry
+  - response
+  - retry
+  - success
+  destination id: 8543b2e2-a373-4d8d-8710-ac314666696b
+  max work queue size: 0
+  max work queue data size: 10000 B
+  flowfile expiration: 0 seconds
+  queue prioritizer class: ''
+- id: 0f292c74-cdbb-4e69-8a26-c834569366c1
+  name: UpdateAttribute/success/LogAttribute
+  source id: 4f8a0c9b-6f7c-458f-9b4d-480addb2e0af
+  source relationship names:
+  - success
+  destination id: 8543b2e2-a373-4d8d-8710-ac314666696b
+  max work queue size: 0
+  max work queue data size: 10000 B
+  flowfile expiration: 0 seconds
+  queue prioritizer class: ''
+Remote Process Groups: []
+NiFi Properties Overrides: {}
diff --git a/libminifi/test/resources/TestInvokeHTTPPostSecure.yml b/libminifi/test/resources/TestInvokeHTTPPostSecure.yml
new file mode 100644
index 0000000..2fbcd1b
--- /dev/null
+++ b/libminifi/test/resources/TestInvokeHTTPPostSecure.yml
@@ -0,0 +1,175 @@
+MiNiFi Config Version: 3
+Flow Controller:
+  name: c++lw
+  comment: Created by MiNiFi C2 Flow Designer
+Core Properties:
+  flow controller graceful shutdown period: 10 sec
+  flow service write delay interval: 500 ms
+  administrative yield duration: 30 sec
+  bored yield duration: 10 millis
+  max concurrent threads: 1
+  variable registry properties: ''
+FlowFile Repository:
+  partitions: 256
+  checkpoint interval: 2 mins
+  always sync: false
+  Swap:
+    threshold: 20000
+    in period: 5 sec
+    in threads: 1
+    out period: 5 sec
+    out threads: 4
+Content Repository:
+  content claim max appendable size: 10 MB
+  content claim max flow files: 100
+  always sync: false
+Provenance Repository:
+  provenance rollover time: 1 min
+  implementation: org.apache.nifi.provenance.MiNiFiPersistentProvenanceRepository
+Component Status Repository:
+  buffer size: 1440
+  snapshot frequency: 1 min
+Security Properties:
+  keystore: ''
+  keystore type: ''
+  keystore password: ''
+  key password: ''
+  truststore: ''
+  truststore type: ''
+  truststore password: ''
+  ssl protocol: ''
+  Sensitive Props:
+    key:
+    algorithm: PBEWITHMD5AND256BITAES-CBC-OPENSSL
+    provider: BC
+Processors:
+- id: 4ed2d51d-076a-49b0-88de-5cf5adf52a7e
+  name: GenerateFlowFile
+  class: org.apache.nifi.minifi.processors.GenerateFlowFile
+  max concurrent tasks: 1
+  scheduling strategy: TIMER_DRIVEN
+  scheduling period: 15000 ms
+  penalization period: 1000 ms
+  yield period: 1000 ms
+  run duration nanos: 0
+  auto-terminated relationships list: []
+  Properties:
+    Batch Size: '1'
+    Data Format: Binary
+    File Size: 1 kB
+    Unique FlowFiles: 'true'
+- id: 1d51724d-dd76-46a0-892d-a7c7408d58dd
+  name: InvokeHTTP
+  class: org.apache.nifi.minifi.processors.InvokeHTTP
+  max concurrent tasks: 1
+  scheduling strategy: TIMER_DRIVEN
+  scheduling period: 1000 ms
+  penalization period: 1000 ms
+  yield period: 1000 ms
+  run duration nanos: 0
+  auto-terminated relationships list: []
+  Properties:
+    Always Output Response: 'false'
+    Connection Timeout: 2 s
+    Content-type: application/octet-stream
+    Disable Peer Verification: 'true'
+    HTTP Method: POST
+    Read Timeout: 3 s
+    Remote URL: https://localhost:0/minifi
+    Use Chunked Encoding: 'false'
+    send-message-body: 'true'
+    SSL Context Service: SSLContextService
+- id: 8543b2e2-a373-4d8d-8710-ac314666696b
+  name: LogAttribute
+  class: org.apache.nifi.minifi.processors.LogAttribute
+  max concurrent tasks: 1
+  scheduling strategy: TIMER_DRIVEN
+  scheduling period: 1000 ms
+  penalization period: 30000 ms
+  yield period: 1000 ms
+  run duration nanos: 0
+  auto-terminated relationships list:
+  - success
+  Properties:
+    FlowFiles To Log: '1'
+    Hexencode Payload: 'false'
+    Log Payload: 'false'
+    Maximum Payload Line Length: '80'
+- id: 4f8a0c9b-6f7c-458f-9b4d-480addb2e0af
+  name: UpdateAttribute
+  class: org.apache.nifi.minifi.processors.UpdateAttribute
+  max concurrent tasks: 1
+  scheduling strategy: TIMER_DRIVEN
+  scheduling period: 200 ms
+  penalization period: 1000 ms
+  yield period: 1000 ms
+  run duration nanos: 0
+  auto-terminated relationships list:
+  - failure
+  Properties:
+    invoke_http: failure
+Controller Services: []
+Process Groups: []
+Input Ports: []
+Output Ports: []
+Funnels: []
+Connections:
+- id: 189bee81-3e56-45ac-b6bb-aa70b7180dc8
+  name: GenerateFlowFile/success/InvokeHTTP
+  source id: 4ed2d51d-076a-49b0-88de-5cf5adf52a7e
+  source relationship names:
+  - success
+  destination id: 1d51724d-dd76-46a0-892d-a7c7408d58dd
+  max work queue size: 0
+  max work queue data size: 10000 B
+  flowfile expiration: 0 seconds
+  queue prioritizer class: ''
+- id: f5d1dd2d-f675-41bd-bf1b-b571aa23ebfb
+  name: InvokeHTTP/failure/UpdateAttribute
+  source id: 1d51724d-dd76-46a0-892d-a7c7408d58dd
+  source relationship names:
+  - failure
+  destination id: 4f8a0c9b-6f7c-458f-9b4d-480addb2e0af
+  max work queue size: 0
+  max work queue data size: 10000 B
+  flowfile expiration: 0 seconds
+  queue prioritizer class: ''
+- id: 000035d8-46a0-460a-809c-f0c6320d532e
+  name: InvokeHTTP/response/LogAttribute
+  source id: 1d51724d-dd76-46a0-892d-a7c7408d58dd
+  source relationship names:
+  - no retry
+  - response
+  - retry
+  - success
+  destination id: 8543b2e2-a373-4d8d-8710-ac314666696b
+  max work queue size: 0
+  max work queue data size: 10000 B
+  flowfile expiration: 0 seconds
+  queue prioritizer class: ''
+- id: 0f292c74-cdbb-4e69-8a26-c834569366c1
+  name: UpdateAttribute/success/LogAttribute
+  source id: 4f8a0c9b-6f7c-458f-9b4d-480addb2e0af
+  source relationship names:
+  - success
+  destination id: 8543b2e2-a373-4d8d-8710-ac314666696b
+  max work queue size: 0
+  max work queue data size: 10000 B
+  flowfile expiration: 0 seconds
+  queue prioritizer class: ''
+  
+Controller Services:
+  - name: SSLContextService
+    id: 2438e3c8-015a-1000-79ca-83af40ec1994
+    class: SSLContextService
+    Properties:
+      Client Certificate:
+          - value: cn.crt.pem
+      Private Key:
+          - value: cn.ckey.pem
+      Passphrase:
+          - value: cn.pass
+      CA Certificate:
+          - value: nifi-cert.pem
+Remote Process Groups: []
+NiFi Properties Overrides: {}