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/08/24 18:13:08 UTC

[GitHub] [nifi-minifi-cpp] fgerlits commented on a diff in pull request #1383: MINIFICPP-1875 HTTPClient should be reusable

fgerlits commented on code in PR #1383:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1383#discussion_r953687253


##########
extensions/http-curl/client/HTTPClient.h:
##########
@@ -81,103 +83,72 @@ class HTTPClient : public BaseHTTPClient, public core::Connectable {
 
   static int debug_callback(CURL *handle, curl_infotype type, char *data, size_t size, void *userptr);
 
-  void setVerbose(bool use_stderr = false) override;
+  void setVerbose(bool use_stderr) override;
 
-  void addFormPart(const std::string& content_type, const std::string& name, HTTPUploadCallback* read_callback, const std::optional<std::string>& filename = std::nullopt);
+  void addFormPart(const std::string& content_type, const std::string& name, std::unique_ptr<utils::HTTPUploadCallback> form_callback, const std::optional<std::string>& filename);
 
   void forceClose();
 
-  void initialize(const std::string &method, const std::string url = "", const std::shared_ptr<minifi::controllers::SSLContextService> ssl_context_service = nullptr) 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) 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) void setReadTimeout(int64_t timeout) override;
+  void initialize(std::string method, std::string url, std::shared_ptr<minifi::controllers::SSLContextService> ssl_context_service) override;
 
   void setConnectionTimeout(std::chrono::milliseconds timeout) override;
 
   void setReadTimeout(std::chrono::milliseconds timeout) override;
 
-  void setUploadCallback(HTTPUploadCallback *callbackObj) override;
+  void setUploadCallback(std::unique_ptr<utils::HTTPUploadCallback> callback) override;
 
-  void setSeekFunction(HTTPUploadCallback *callbackObj) override;
+  virtual void setReadCallback(std::unique_ptr<utils::HTTPReadCallback> callback);

Review Comment:
   does this need to be virtual?  I can't see any subclasses of `HTTPClient`



##########
extensions/http-curl/client/HTTPClient.cpp:
##########
@@ -163,212 +160,228 @@ bool HTTPClient::setSpecificSSLVersion(SSLVersion specific_version) {
 }
 
 // If not set, the default will be TLS 1.0, see https://curl.haxx.se/libcurl/c/CURLOPT_SSLVERSION.html
-bool HTTPClient::setMinimumSSLVersion(SSLVersion minimum_version) {
+bool HTTPClient::setMinimumSSLVersion(utils::SSLVersion minimum_version) {
   CURLcode ret = CURLE_UNKNOWN_OPTION;
   switch (minimum_version) {
-    case SSLVersion::TLSv1_0:
-      ret = curl_easy_setopt(http_session_, CURLOPT_SSLVERSION, CURL_SSLVERSION_TLSv1_0);
+    case utils::SSLVersion::TLSv1_0:
+      ret = curl_easy_setopt(http_session_.get(), CURLOPT_SSLVERSION, CURL_SSLVERSION_TLSv1_0);
       break;
-    case SSLVersion::TLSv1_1:
-      ret = curl_easy_setopt(http_session_, CURLOPT_SSLVERSION, CURL_SSLVERSION_TLSv1_1);
+    case utils::SSLVersion::TLSv1_1:
+      ret = curl_easy_setopt(http_session_.get(), CURLOPT_SSLVERSION, CURL_SSLVERSION_TLSv1_1);
       break;
-    case SSLVersion::TLSv1_2:
-      ret = curl_easy_setopt(http_session_, CURLOPT_SSLVERSION, CURL_SSLVERSION_TLSv1_2);
+    case utils::SSLVersion::TLSv1_2:
+      ret = curl_easy_setopt(http_session_.get(), CURLOPT_SSLVERSION, CURL_SSLVERSION_TLSv1_2);
       break;
   }
 
   return ret == CURLE_OK;
 }
 
-DEPRECATED(/*deprecated in*/ 0.8.0, /*will remove in */ 2.0) void HTTPClient::setConnectionTimeout(int64_t timeout) {
-  setConnectionTimeout(std::chrono::milliseconds(timeout * 1000));
-}
-
-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::setKeepAliveProbe(std::optional<KeepAliveProbeData> probe_data) {
+  if (probe_data) {
+    curl_easy_setopt(http_session_.get(), CURLOPT_TCP_KEEPALIVE, true);
+    curl_easy_setopt(http_session_.get(), CURLOPT_TCP_KEEPINTVL, probe_data->keep_alive_interval.count());
+    curl_easy_setopt(http_session_.get(), CURLOPT_TCP_KEEPIDLE, probe_data->keep_alive_delay.count());
+  } else {
+    curl_easy_setopt(http_session_.get(), CURLOPT_TCP_KEEPALIVE, false);
+  }
 }
 
 void HTTPClient::setConnectionTimeout(std::chrono::milliseconds timeout) {
-  connect_timeout_ms_ = timeout;
+  if (timeout < 0ms) {
+    logger_->log_error("Invalid timeout");

Review Comment:
   We could make our future selves' life easier by adding more info, eg. "Invalid HTTP connection timeout: -123 ms" / "Invalid HTTP read timeout: -345 ms".



##########
extensions/http-curl/client/HTTPClient.cpp:
##########
@@ -163,212 +160,228 @@ bool HTTPClient::setSpecificSSLVersion(SSLVersion specific_version) {
 }
 
 // If not set, the default will be TLS 1.0, see https://curl.haxx.se/libcurl/c/CURLOPT_SSLVERSION.html
-bool HTTPClient::setMinimumSSLVersion(SSLVersion minimum_version) {
+bool HTTPClient::setMinimumSSLVersion(utils::SSLVersion minimum_version) {
   CURLcode ret = CURLE_UNKNOWN_OPTION;
   switch (minimum_version) {
-    case SSLVersion::TLSv1_0:
-      ret = curl_easy_setopt(http_session_, CURLOPT_SSLVERSION, CURL_SSLVERSION_TLSv1_0);
+    case utils::SSLVersion::TLSv1_0:
+      ret = curl_easy_setopt(http_session_.get(), CURLOPT_SSLVERSION, CURL_SSLVERSION_TLSv1_0);
       break;
-    case SSLVersion::TLSv1_1:
-      ret = curl_easy_setopt(http_session_, CURLOPT_SSLVERSION, CURL_SSLVERSION_TLSv1_1);
+    case utils::SSLVersion::TLSv1_1:
+      ret = curl_easy_setopt(http_session_.get(), CURLOPT_SSLVERSION, CURL_SSLVERSION_TLSv1_1);
       break;
-    case SSLVersion::TLSv1_2:
-      ret = curl_easy_setopt(http_session_, CURLOPT_SSLVERSION, CURL_SSLVERSION_TLSv1_2);
+    case utils::SSLVersion::TLSv1_2:
+      ret = curl_easy_setopt(http_session_.get(), CURLOPT_SSLVERSION, CURL_SSLVERSION_TLSv1_2);
       break;
   }
 
   return ret == CURLE_OK;
 }
 
-DEPRECATED(/*deprecated in*/ 0.8.0, /*will remove in */ 2.0) void HTTPClient::setConnectionTimeout(int64_t timeout) {
-  setConnectionTimeout(std::chrono::milliseconds(timeout * 1000));
-}
-
-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::setKeepAliveProbe(std::optional<KeepAliveProbeData> probe_data) {
+  if (probe_data) {
+    curl_easy_setopt(http_session_.get(), CURLOPT_TCP_KEEPALIVE, true);
+    curl_easy_setopt(http_session_.get(), CURLOPT_TCP_KEEPINTVL, probe_data->keep_alive_interval.count());
+    curl_easy_setopt(http_session_.get(), CURLOPT_TCP_KEEPIDLE, probe_data->keep_alive_delay.count());
+  } else {
+    curl_easy_setopt(http_session_.get(), CURLOPT_TCP_KEEPALIVE, false);
+  }
 }
 
 void HTTPClient::setConnectionTimeout(std::chrono::milliseconds timeout) {
-  connect_timeout_ms_ = timeout;
+  if (timeout < 0ms) {
+    logger_->log_error("Invalid timeout");
+    return;
+  }
+  connect_timeout_ = timeout;
 }
 
 void HTTPClient::setReadTimeout(std::chrono::milliseconds timeout) {
-  read_timeout_ms_ = timeout;
+  if (timeout < 0ms) {
+    logger_->log_error("Invalid timeout");
+    return;
+  }
+  read_timeout_ = timeout;
 }
 
-void HTTPClient::setReadCallback(HTTPReadCallback *callbackObj) {
-  callback = callbackObj;
-  curl_easy_setopt(http_session_, CURLOPT_WRITEFUNCTION, &utils::HTTPRequestResponse::recieve_write);
-  curl_easy_setopt(http_session_, CURLOPT_WRITEDATA, static_cast<void*>(callbackObj));
+void HTTPClient::setReadCallback(std::unique_ptr<utils::HTTPReadCallback> callback) {
+  read_callback_ = std::move(callback);
+  curl_easy_setopt(http_session_.get(), CURLOPT_WRITEFUNCTION, &utils::HTTPRequestResponse::recieve_write);
+  curl_easy_setopt(http_session_.get(), CURLOPT_WRITEDATA, static_cast<void*>(read_callback_.get()));
 }
 
-void HTTPClient::setUploadCallback(HTTPUploadCallback *callbackObj) {
+void HTTPClient::setUploadCallback(std::unique_ptr<utils::HTTPUploadCallback> callback) {
   logger_->log_debug("Setting callback for %s", url_);
-  write_callback_ = callbackObj;
-  if (method_ == "put" || method_ == "PUT") {
-    curl_easy_setopt(http_session_, CURLOPT_INFILESIZE_LARGE, (curl_off_t) callbackObj->ptr->getBufferSize());
+  write_callback_ = std::move(callback);
+  if (method_ == "PUT") {
+    curl_easy_setopt(http_session_.get(), CURLOPT_INFILESIZE_LARGE, (curl_off_t) write_callback_->getBufferSize());
   }
-  curl_easy_setopt(http_session_, CURLOPT_READFUNCTION, &utils::HTTPRequestResponse::send_write);
-  curl_easy_setopt(http_session_, CURLOPT_READDATA, static_cast<void*>(callbackObj));
-}
-
-void HTTPClient::setSeekFunction(HTTPUploadCallback *callbackObj) {
-  curl_easy_setopt(http_session_, CURLOPT_SEEKDATA, static_cast<void*>(callbackObj));
-  curl_easy_setopt(http_session_, CURLOPT_SEEKFUNCTION, &utils::HTTPRequestResponse::seek_callback);
+  curl_easy_setopt(http_session_.get(), CURLOPT_READFUNCTION, &utils::HTTPRequestResponse::send_write);
+  curl_easy_setopt(http_session_.get(), CURLOPT_READDATA, static_cast<void*>(write_callback_.get()));
+  curl_easy_setopt(http_session_.get(), CURLOPT_SEEKDATA, static_cast<void*>(write_callback_.get()));
+  curl_easy_setopt(http_session_.get(), CURLOPT_SEEKFUNCTION, &utils::HTTPRequestResponse::seek_callback);
 }
 
 void HTTPClient::setContentType(std::string content_type) {
-  content_type_ = "Content-Type: " + content_type;
-  headers_ = curl_slist_append(headers_, content_type_.c_str());
+  request_headers_["Content-Type"] = std::move(content_type);
 }
 
 std::string HTTPClient::escape(std::string string_to_escape) {
   struct curl_deleter { void operator()(void* p) noexcept { curl_free(p); } };
-  std::unique_ptr<char, curl_deleter> escaped_chars{curl_easy_escape(http_session_, string_to_escape.c_str(), gsl::narrow<int>(string_to_escape.length()))};
+  std::unique_ptr<char, curl_deleter> escaped_chars{curl_easy_escape(http_session_.get(), string_to_escape.c_str(), gsl::narrow<int>(string_to_escape.length()))};
   std::string escaped_string(escaped_chars.get());
   return escaped_string;
 }
 
 void HTTPClient::setPostFields(const std::string& input) {
-  curl_easy_setopt(http_session_, CURLOPT_POSTFIELDSIZE, input.length());
-  curl_easy_setopt(http_session_, CURLOPT_COPYPOSTFIELDS, input.c_str());
+  curl_easy_setopt(http_session_.get(), CURLOPT_POSTFIELDSIZE, input.length());
+  curl_easy_setopt(http_session_.get(), CURLOPT_COPYPOSTFIELDS, input.c_str());
 }
 
 void HTTPClient::setPostSize(size_t size) {
-  curl_easy_setopt(http_session_, CURLOPT_POSTFIELDSIZE, size);
+  curl_easy_setopt(http_session_.get(), CURLOPT_POSTFIELDSIZE, size);
 }
 
-void HTTPClient::setHeaders(struct curl_slist *list) {
-  headers_ = list;
+void HTTPClient::setHTTPProxy(const utils::HTTPProxy &proxy) {
+  if (!proxy.host.empty()) {
+    curl_easy_setopt(http_session_.get(), CURLOPT_PROXY, proxy.host.c_str());
+    curl_easy_setopt(http_session_.get(), CURLOPT_PROXYPORT, proxy.port);
+    if (!proxy.username.empty()) {
+      curl_easy_setopt(http_session_.get(), CURLOPT_PROXYAUTH, CURLAUTH_ANY);
+      std::string value = proxy.username + ":" + proxy.password;
+      curl_easy_setopt(http_session_.get(), CURLOPT_PROXYUSERPWD, value.c_str());
+    }
+  }
 }
 
-void HTTPClient::appendHeader(const std::string &new_header) {
-  headers_ = curl_slist_append(headers_, new_header.c_str());
+void HTTPClient::setRequestHeader(std::string key, std::optional<std::string> value) {
+  if (value)
+    request_headers_[std::move(key)] = std::move(*value);
+  else
+    request_headers_.erase(key);
 }
 
-void HTTPClient::appendHeader(const std::string &key, const std::string &value) {
-  headers_ = curl_slist_append(headers_, utils::StringUtils::join_pack(key, ": ", value).c_str());
-}
+namespace {
+struct CurlSListFreeAll {
+  void operator()(struct curl_slist* slist) const {
+    curl_slist_free_all(slist);
+  }
+};
+
+std::unique_ptr<struct curl_slist, CurlSListFreeAll> getCurlSList(const std::unordered_map<std::string, std::string>& request_headers) {
+  curl_slist* new_list = nullptr;
+  for (const auto& [header_key, header_value] : request_headers)
+    new_list = curl_slist_append(new_list, utils::StringUtils::join_pack(header_key, ": ", header_value).c_str());
 
-void HTTPClient::setUseChunkedEncoding() {
-  headers_ = curl_slist_append(headers_, "Transfer-Encoding: chunked");
+  return {new_list, {}};
 }
+}  // namespace
+
 
 bool HTTPClient::submit() {
-  if (IsNullOrEmpty(url_))
+  if (url_.empty()) {
+    logger_->log_error("Tried to submit to an empty url");
     return false;
+  }
 
-  int absoluteTimeout = std::max(0, 3 * static_cast<int>(read_timeout_ms_.count()));
+  response_data_.clear();
 
-  curl_easy_setopt(http_session_, CURLOPT_NOSIGNAL, 1);
-  // setting it to 0 will result in the default 300 second timeout
-  curl_easy_setopt(http_session_, CURLOPT_CONNECTTIMEOUT_MS, std::max(0, static_cast<int>(connect_timeout_ms_.count())));
-  curl_easy_setopt(http_session_, CURLOPT_TIMEOUT_MS, absoluteTimeout);
+  curl_easy_setopt(http_session_.get(), CURLOPT_NOSIGNAL, 1);
+  curl_easy_setopt(http_session_.get(), CURLOPT_CONNECTTIMEOUT_MS, connect_timeout_.count());
+  curl_easy_setopt(http_session_.get(), CURLOPT_TIMEOUT_MS, getAbsoluteTimeout().count());
 
-  if (read_timeout_ms_.count() > 0) {
+  if (read_timeout_ > 0ms) {
     progress_.reset();
-    curl_easy_setopt(http_session_, CURLOPT_NOPROGRESS, 0);
-    curl_easy_setopt(http_session_, CURLOPT_XFERINFOFUNCTION, onProgress);
-    curl_easy_setopt(http_session_, CURLOPT_XFERINFODATA, this);
+    curl_easy_setopt(http_session_.get(), CURLOPT_NOPROGRESS, 0);
+    curl_easy_setopt(http_session_.get(), CURLOPT_XFERINFOFUNCTION, onProgress);
+    curl_easy_setopt(http_session_.get(), CURLOPT_XFERINFODATA, this);
   } else {
     // the user explicitly set it to 0
-    curl_easy_setopt(http_session_, CURLOPT_NOPROGRESS, 1);
-  }
-  if (headers_ != nullptr) {
-    headers_ = curl_slist_append(headers_, "Expect:");
-    curl_easy_setopt(http_session_, CURLOPT_HTTPHEADER, headers_);
+    curl_easy_setopt(http_session_.get(), CURLOPT_NOPROGRESS, 1);
   }
 
-  if (username_password_) {
-    curl_easy_setopt(http_session_, CURLOPT_USERNAME, username_password_->username.c_str());
-    curl_easy_setopt(http_session_, CURLOPT_PASSWORD, username_password_->password.c_str());
+  auto headers = getCurlSList(request_headers_);
+  if (headers) {
+    curl_slist_append(headers.get(), "Expect:");
+    curl_easy_setopt(http_session_.get(), CURLOPT_HTTPHEADER, headers.get());
   }
 
-  curl_easy_setopt(http_session_, CURLOPT_URL, url_.c_str());
+  curl_easy_setopt(http_session_.get(), CURLOPT_URL, url_.c_str());
   logger_->log_debug("Submitting to %s", url_);
-  if (callback == nullptr) {
-    content_.ptr = &read_callback_;
-    curl_easy_setopt(http_session_, CURLOPT_WRITEFUNCTION, &utils::HTTPRequestResponse::recieve_write);
-    curl_easy_setopt(http_session_, CURLOPT_WRITEDATA, static_cast<void*>(&content_));
-  }
-  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_.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, keepAlive.count());
-    curl_easy_setopt(http_session_, CURLOPT_TCP_KEEPIDLE, keepIdle.count());
-  } else {
-    logger_->log_debug("Not using keep alive");
-    curl_easy_setopt(http_session_, CURLOPT_TCP_KEEPALIVE, 0L);
+  if (read_callback_ == nullptr) {
+    curl_easy_setopt(http_session_.get(), CURLOPT_WRITEFUNCTION, &utils::HTTPRequestResponse::recieve_write);
+    curl_easy_setopt(http_session_.get(), CURLOPT_WRITEDATA, static_cast<void*>(&content_));
   }
+
+  curl_easy_setopt(http_session_.get(), CURLOPT_HEADERFUNCTION, &utils::HTTPHeaderResponse::receive_headers);
+  curl_easy_setopt(http_session_.get(), CURLOPT_HEADERDATA, static_cast<void*>(&response_data_.header_response));
+
   if (form_ != nullptr) {
-    curl_easy_setopt(http_session_, CURLOPT_MIMEPOST, form_);
-  }
-  res = curl_easy_perform(http_session_);
-  if (callback == nullptr) {
-    read_callback_.close();
-  }
-  long http_code;  // NOLINT long due to libcurl API
-  curl_easy_getinfo(http_session_, CURLINFO_RESPONSE_CODE, &http_code);
-  http_code_ = http_code;
-  curl_easy_getinfo(http_session_, CURLINFO_CONTENT_TYPE, &content_type_str_);
-  if (res == CURLE_OPERATION_TIMEDOUT) {
-    logger_->log_error("HTTP operation timed out, with absolute timeout %dms\n", absoluteTimeout);
-  }
-  if (res != CURLE_OK) {
-    logger_->log_error("curl_easy_perform() failed %s on %s, error code %d\n", curl_easy_strerror(res), url_, res);
+    curl_easy_setopt(http_session_.get(), CURLOPT_MIMEPOST, form_.get());
+  }
+  res_ = curl_easy_perform(http_session_.get());
+  if (read_callback_ == nullptr) {
+    content_.close();
+  }
+  long http_code;  // NOLINT(runtime/int) long due to libcurl API
+  curl_easy_getinfo(http_session_.get(), CURLINFO_RESPONSE_CODE, &http_code);
+  response_data_.response_code = http_code;
+  curl_easy_getinfo(http_session_.get(), CURLINFO_CONTENT_TYPE, &response_data_.response_content_type);
+  if (res_ == CURLE_OPERATION_TIMEDOUT) {
+    logger_->log_error("HTTP operation timed out, with absolute timeout %" PRId64 "ms\n", getAbsoluteTimeout().count());
+  }
+  if (res_ != CURLE_OK) {
+    logger_->log_info("%d", request_headers_.size());
+    logger_->log_error("curl_easy_perform() failed %s on %s, error code %d\n", curl_easy_strerror(res_), url_, res_);
     return false;
   }
 
   logger_->log_debug("Finished with %s", url_);
   return true;
 }
 
-CURLcode HTTPClient::getResponseResult() {
-  return res;
-}
-
 int64_t HTTPClient::getResponseCode() const {
-  return http_code_;
+  return response_data_.response_code;
 }
 
 const char *HTTPClient::getContentType() {
-  return content_type_str_;
+  return response_data_.response_content_type;
 }
 
 const std::vector<char> &HTTPClient::getResponseBody() {
-  if (response_body_.empty()) {
-    if (callback && callback->ptr) {
-      response_body_ = callback->ptr->to_string();
+  if (response_data_.response_body.empty()) {
+    if (read_callback_) {
+      response_data_.response_body = read_callback_->to_string();
     } else {
-      response_body_ = read_callback_.to_string();
+      response_data_.response_body = content_.to_string();
     }
   }
-  return response_body_;
-}
-
-void HTTPClient::set_request_method(const std::string method) {
-  std::string my_method = method;
-  std::transform(my_method.begin(), my_method.end(), my_method.begin(), ::toupper);
-  if (my_method == "POST") {
-    curl_easy_setopt(http_session_, CURLOPT_POST, 1L);
-  } else if (my_method == "PUT") {
-    curl_easy_setopt(http_session_, CURLOPT_UPLOAD, 1L);
-  } else if (my_method == "HEAD") {
-    curl_easy_setopt(http_session_, CURLOPT_NOBODY, 1L);
-  } else if (my_method == "GET") {
+  return response_data_.response_body;
+}
+
+void HTTPClient::set_request_method(std::string method) {
+  if (method_ == method)
+    return;
+  method_ = std::move(method);
+  std::transform(method_.begin(), method_.end(), method_.begin(), ::toupper);

Review Comment:
   two nitpicking comments:
   * we could transform `method` to uppercase first, and then compare it to `method_`, otherwise if you call `set_request_method("put")` 100 times, then `curl_easy_setopt` will be called 100 times;
   * in theory, `std::transform(method_.begin(), method_.end(), method_.begin(), [](auto ch) { return ::toupper(static_cast<unsigned char>(ch)); })` is more correct -- although it's less readable, and the version without the cast probably won't cause any issues, so your call.



##########
extensions/http-curl/client/HTTPStream.cpp:
##########
@@ -26,55 +26,55 @@
 #include "io/validation.h"
 #include "utils/gsl.h"
 
-namespace org {
-namespace apache {
-namespace nifi {
-namespace minifi {
-namespace io {
+namespace org::apache::nifi::minifi::extensions::curl {
 
-HttpStream::HttpStream(std::shared_ptr<utils::HTTPClient> client)
+HttpStream::HttpStream(std::shared_ptr<HTTPClient> client)
     : http_client_(std::move(client)),
       written(0),
-      // given the nature of the stream we don't want to slow libCURL, we will produce
-      // a warning instead allowing us to adjust it server side or through the local configuration.
-      http_read_callback_(66560, true),
+    // given the nature of the stream we don't want to slow libCURL, we will produce
+    // a warning instead allowing us to adjust it server side or through the local configuration.
       started_(false) {
   // submit early on
 }
 
 void HttpStream::close() {
-  http_callback_.close();
-  http_read_callback_.close();
+  if (auto read_callback = http_client_->getReadCallback())
+    read_callback->close();
+  if (auto upload_callback = http_client_->getUploadCallback())
+    upload_callback->close();
 }
 
 void HttpStream::seek(size_t /*offset*/) {
   // seek is an unnecessary part of this implementation
   throw std::logic_error{"HttpStream::seek is unimplemented"};
 }
 
-size_t HttpStream::tell() const  {
+size_t HttpStream::tell() const {
   // tell is an unnecessary part of this implementation
   throw std::logic_error{"HttpStream::tell is unimplemented"};
 }
 
 // data stream overrides
 
-size_t HttpStream::write(const uint8_t *value, size_t size) {
+size_t HttpStream::write(const uint8_t* value, size_t size) {
   if (size == 0) return 0;
   if (IsNullOrEmpty(value)) {
-    return STREAM_ERROR;
+    return io::STREAM_ERROR;
   }
   if (!started_) {
     std::lock_guard<std::mutex> lock(mutex_);
     if (!started_) {
-      callback_.ptr = &http_callback_;
-      callback_.pos = 0;
-      http_client_->setUploadCallback(&callback_);
+      auto callback = std::make_unique<HttpStreamingCallback>();
+      callback->pos = 0;

Review Comment:
   since we are creating a new callback object now, there is no need to set `pos = 0` any longer (also at line 88)



##########
libminifi/include/utils/BaseHTTPClient.h:
##########
@@ -51,38 +39,22 @@ struct HTTPProxy {
   int port = 0;
 };
 
-struct HTTPUploadCallback {
-  HTTPUploadCallback() {
-    stop = false;
-    ptr = nullptr;
-    pos = 0;
-  }
-  std::mutex mutex;
-  std::atomic<bool> stop;
-  ByteInputCallback *ptr;
-  size_t pos;
-
-  size_t getPos() {
-    std::lock_guard<std::mutex> lock(mutex);
-    return pos;
-  }
+class HTTPUploadCallback : public ByteInputCallback {
+ public:
+  template<typename... Args>
+  explicit HTTPUploadCallback(Args&& ... args) : ByteInputCallback(std::forward<Args>(args)...) {}

Review Comment:
   I think `using ByteInputCallback::ByteInputCallback`  would do the same, and is simpler (also in `HTTPReadCallback`)



##########
extensions/http-curl/client/HTTPCallback.h:
##########
@@ -53,19 +50,12 @@ namespace utils {
  *  - because of this, all functions that request data at a specific offset are implicit seeks and potentially modify
  *    the current buffer
  */
-class HttpStreamingCallback final : public ByteInputCallback {
+class HttpStreamingCallback final : public utils::HTTPUploadCallback {
  public:
-  HttpStreamingCallback()
-      : is_alive_(true),
-        total_bytes_loaded_(0U),
-        current_buffer_start_(0U),
-        current_pos_(0U),
-        ptr_(nullptr) {
-  }
-
+  HttpStreamingCallback() = default;
   ~HttpStreamingCallback() override = default;

Review Comment:
   could we remove the constructor and destructor and let the compiler auto-generate them?



##########
extensions/http-curl/client/HTTPClient.cpp:
##########
@@ -163,212 +160,228 @@ bool HTTPClient::setSpecificSSLVersion(SSLVersion specific_version) {
 }
 
 // If not set, the default will be TLS 1.0, see https://curl.haxx.se/libcurl/c/CURLOPT_SSLVERSION.html
-bool HTTPClient::setMinimumSSLVersion(SSLVersion minimum_version) {
+bool HTTPClient::setMinimumSSLVersion(utils::SSLVersion minimum_version) {
   CURLcode ret = CURLE_UNKNOWN_OPTION;
   switch (minimum_version) {
-    case SSLVersion::TLSv1_0:
-      ret = curl_easy_setopt(http_session_, CURLOPT_SSLVERSION, CURL_SSLVERSION_TLSv1_0);
+    case utils::SSLVersion::TLSv1_0:
+      ret = curl_easy_setopt(http_session_.get(), CURLOPT_SSLVERSION, CURL_SSLVERSION_TLSv1_0);
       break;
-    case SSLVersion::TLSv1_1:
-      ret = curl_easy_setopt(http_session_, CURLOPT_SSLVERSION, CURL_SSLVERSION_TLSv1_1);
+    case utils::SSLVersion::TLSv1_1:
+      ret = curl_easy_setopt(http_session_.get(), CURLOPT_SSLVERSION, CURL_SSLVERSION_TLSv1_1);
       break;
-    case SSLVersion::TLSv1_2:
-      ret = curl_easy_setopt(http_session_, CURLOPT_SSLVERSION, CURL_SSLVERSION_TLSv1_2);
+    case utils::SSLVersion::TLSv1_2:
+      ret = curl_easy_setopt(http_session_.get(), CURLOPT_SSLVERSION, CURL_SSLVERSION_TLSv1_2);
       break;
   }
 
   return ret == CURLE_OK;
 }
 
-DEPRECATED(/*deprecated in*/ 0.8.0, /*will remove in */ 2.0) void HTTPClient::setConnectionTimeout(int64_t timeout) {
-  setConnectionTimeout(std::chrono::milliseconds(timeout * 1000));
-}
-
-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::setKeepAliveProbe(std::optional<KeepAliveProbeData> probe_data) {
+  if (probe_data) {
+    curl_easy_setopt(http_session_.get(), CURLOPT_TCP_KEEPALIVE, true);
+    curl_easy_setopt(http_session_.get(), CURLOPT_TCP_KEEPINTVL, probe_data->keep_alive_interval.count());
+    curl_easy_setopt(http_session_.get(), CURLOPT_TCP_KEEPIDLE, probe_data->keep_alive_delay.count());
+  } else {
+    curl_easy_setopt(http_session_.get(), CURLOPT_TCP_KEEPALIVE, false);
+  }
 }
 
 void HTTPClient::setConnectionTimeout(std::chrono::milliseconds timeout) {
-  connect_timeout_ms_ = timeout;
+  if (timeout < 0ms) {
+    logger_->log_error("Invalid timeout");
+    return;
+  }
+  connect_timeout_ = timeout;
 }
 
 void HTTPClient::setReadTimeout(std::chrono::milliseconds timeout) {
-  read_timeout_ms_ = timeout;
+  if (timeout < 0ms) {
+    logger_->log_error("Invalid timeout");
+    return;
+  }
+  read_timeout_ = timeout;
 }
 
-void HTTPClient::setReadCallback(HTTPReadCallback *callbackObj) {
-  callback = callbackObj;
-  curl_easy_setopt(http_session_, CURLOPT_WRITEFUNCTION, &utils::HTTPRequestResponse::recieve_write);
-  curl_easy_setopt(http_session_, CURLOPT_WRITEDATA, static_cast<void*>(callbackObj));
+void HTTPClient::setReadCallback(std::unique_ptr<utils::HTTPReadCallback> callback) {
+  read_callback_ = std::move(callback);
+  curl_easy_setopt(http_session_.get(), CURLOPT_WRITEFUNCTION, &utils::HTTPRequestResponse::recieve_write);

Review Comment:
   this is an old typo, but it hurts my eyes: please change this to `receive_write`



##########
extensions/http-curl/client/HTTPStream.cpp:
##########
@@ -26,55 +26,55 @@
 #include "io/validation.h"
 #include "utils/gsl.h"
 
-namespace org {
-namespace apache {
-namespace nifi {
-namespace minifi {
-namespace io {
+namespace org::apache::nifi::minifi::extensions::curl {
 
-HttpStream::HttpStream(std::shared_ptr<utils::HTTPClient> client)
+HttpStream::HttpStream(std::shared_ptr<HTTPClient> client)
     : http_client_(std::move(client)),
       written(0),
-      // given the nature of the stream we don't want to slow libCURL, we will produce
-      // a warning instead allowing us to adjust it server side or through the local configuration.
-      http_read_callback_(66560, true),
+    // given the nature of the stream we don't want to slow libCURL, we will produce
+    // a warning instead allowing us to adjust it server side or through the local configuration.

Review Comment:
   I think this comment referred to the parameters of `http_read_callback_`.  Is it still useful, or should we remove it (or move it somewhere else)?



##########
extensions/http-curl/client/HTTPClient.cpp:
##########
@@ -163,212 +160,228 @@ bool HTTPClient::setSpecificSSLVersion(SSLVersion specific_version) {
 }
 
 // If not set, the default will be TLS 1.0, see https://curl.haxx.se/libcurl/c/CURLOPT_SSLVERSION.html
-bool HTTPClient::setMinimumSSLVersion(SSLVersion minimum_version) {
+bool HTTPClient::setMinimumSSLVersion(utils::SSLVersion minimum_version) {
   CURLcode ret = CURLE_UNKNOWN_OPTION;
   switch (minimum_version) {
-    case SSLVersion::TLSv1_0:
-      ret = curl_easy_setopt(http_session_, CURLOPT_SSLVERSION, CURL_SSLVERSION_TLSv1_0);
+    case utils::SSLVersion::TLSv1_0:
+      ret = curl_easy_setopt(http_session_.get(), CURLOPT_SSLVERSION, CURL_SSLVERSION_TLSv1_0);
       break;
-    case SSLVersion::TLSv1_1:
-      ret = curl_easy_setopt(http_session_, CURLOPT_SSLVERSION, CURL_SSLVERSION_TLSv1_1);
+    case utils::SSLVersion::TLSv1_1:
+      ret = curl_easy_setopt(http_session_.get(), CURLOPT_SSLVERSION, CURL_SSLVERSION_TLSv1_1);
       break;
-    case SSLVersion::TLSv1_2:
-      ret = curl_easy_setopt(http_session_, CURLOPT_SSLVERSION, CURL_SSLVERSION_TLSv1_2);
+    case utils::SSLVersion::TLSv1_2:
+      ret = curl_easy_setopt(http_session_.get(), CURLOPT_SSLVERSION, CURL_SSLVERSION_TLSv1_2);
       break;
   }
 
   return ret == CURLE_OK;
 }
 
-DEPRECATED(/*deprecated in*/ 0.8.0, /*will remove in */ 2.0) void HTTPClient::setConnectionTimeout(int64_t timeout) {
-  setConnectionTimeout(std::chrono::milliseconds(timeout * 1000));
-}
-
-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::setKeepAliveProbe(std::optional<KeepAliveProbeData> probe_data) {
+  if (probe_data) {
+    curl_easy_setopt(http_session_.get(), CURLOPT_TCP_KEEPALIVE, true);
+    curl_easy_setopt(http_session_.get(), CURLOPT_TCP_KEEPINTVL, probe_data->keep_alive_interval.count());
+    curl_easy_setopt(http_session_.get(), CURLOPT_TCP_KEEPIDLE, probe_data->keep_alive_delay.count());
+  } else {
+    curl_easy_setopt(http_session_.get(), CURLOPT_TCP_KEEPALIVE, false);
+  }
 }
 
 void HTTPClient::setConnectionTimeout(std::chrono::milliseconds timeout) {
-  connect_timeout_ms_ = timeout;
+  if (timeout < 0ms) {
+    logger_->log_error("Invalid timeout");
+    return;
+  }
+  connect_timeout_ = timeout;
 }
 
 void HTTPClient::setReadTimeout(std::chrono::milliseconds timeout) {
-  read_timeout_ms_ = timeout;
+  if (timeout < 0ms) {
+    logger_->log_error("Invalid timeout");
+    return;
+  }
+  read_timeout_ = timeout;
 }
 
-void HTTPClient::setReadCallback(HTTPReadCallback *callbackObj) {
-  callback = callbackObj;
-  curl_easy_setopt(http_session_, CURLOPT_WRITEFUNCTION, &utils::HTTPRequestResponse::recieve_write);
-  curl_easy_setopt(http_session_, CURLOPT_WRITEDATA, static_cast<void*>(callbackObj));
+void HTTPClient::setReadCallback(std::unique_ptr<utils::HTTPReadCallback> callback) {
+  read_callback_ = std::move(callback);
+  curl_easy_setopt(http_session_.get(), CURLOPT_WRITEFUNCTION, &utils::HTTPRequestResponse::recieve_write);
+  curl_easy_setopt(http_session_.get(), CURLOPT_WRITEDATA, static_cast<void*>(read_callback_.get()));
 }
 
-void HTTPClient::setUploadCallback(HTTPUploadCallback *callbackObj) {
+void HTTPClient::setUploadCallback(std::unique_ptr<utils::HTTPUploadCallback> callback) {
   logger_->log_debug("Setting callback for %s", url_);
-  write_callback_ = callbackObj;
-  if (method_ == "put" || method_ == "PUT") {
-    curl_easy_setopt(http_session_, CURLOPT_INFILESIZE_LARGE, (curl_off_t) callbackObj->ptr->getBufferSize());
+  write_callback_ = std::move(callback);
+  if (method_ == "PUT") {
+    curl_easy_setopt(http_session_.get(), CURLOPT_INFILESIZE_LARGE, (curl_off_t) write_callback_->getBufferSize());
   }
-  curl_easy_setopt(http_session_, CURLOPT_READFUNCTION, &utils::HTTPRequestResponse::send_write);
-  curl_easy_setopt(http_session_, CURLOPT_READDATA, static_cast<void*>(callbackObj));
-}
-
-void HTTPClient::setSeekFunction(HTTPUploadCallback *callbackObj) {
-  curl_easy_setopt(http_session_, CURLOPT_SEEKDATA, static_cast<void*>(callbackObj));
-  curl_easy_setopt(http_session_, CURLOPT_SEEKFUNCTION, &utils::HTTPRequestResponse::seek_callback);
+  curl_easy_setopt(http_session_.get(), CURLOPT_READFUNCTION, &utils::HTTPRequestResponse::send_write);
+  curl_easy_setopt(http_session_.get(), CURLOPT_READDATA, static_cast<void*>(write_callback_.get()));
+  curl_easy_setopt(http_session_.get(), CURLOPT_SEEKDATA, static_cast<void*>(write_callback_.get()));
+  curl_easy_setopt(http_session_.get(), CURLOPT_SEEKFUNCTION, &utils::HTTPRequestResponse::seek_callback);
 }
 
 void HTTPClient::setContentType(std::string content_type) {
-  content_type_ = "Content-Type: " + content_type;
-  headers_ = curl_slist_append(headers_, content_type_.c_str());
+  request_headers_["Content-Type"] = std::move(content_type);
 }
 
 std::string HTTPClient::escape(std::string string_to_escape) {
   struct curl_deleter { void operator()(void* p) noexcept { curl_free(p); } };
-  std::unique_ptr<char, curl_deleter> escaped_chars{curl_easy_escape(http_session_, string_to_escape.c_str(), gsl::narrow<int>(string_to_escape.length()))};
+  std::unique_ptr<char, curl_deleter> escaped_chars{curl_easy_escape(http_session_.get(), string_to_escape.c_str(), gsl::narrow<int>(string_to_escape.length()))};
   std::string escaped_string(escaped_chars.get());
   return escaped_string;
 }
 
 void HTTPClient::setPostFields(const std::string& input) {
-  curl_easy_setopt(http_session_, CURLOPT_POSTFIELDSIZE, input.length());
-  curl_easy_setopt(http_session_, CURLOPT_COPYPOSTFIELDS, input.c_str());
+  curl_easy_setopt(http_session_.get(), CURLOPT_POSTFIELDSIZE, input.length());
+  curl_easy_setopt(http_session_.get(), CURLOPT_COPYPOSTFIELDS, input.c_str());
 }
 
 void HTTPClient::setPostSize(size_t size) {
-  curl_easy_setopt(http_session_, CURLOPT_POSTFIELDSIZE, size);
+  curl_easy_setopt(http_session_.get(), CURLOPT_POSTFIELDSIZE, size);
 }
 
-void HTTPClient::setHeaders(struct curl_slist *list) {
-  headers_ = list;
+void HTTPClient::setHTTPProxy(const utils::HTTPProxy &proxy) {
+  if (!proxy.host.empty()) {
+    curl_easy_setopt(http_session_.get(), CURLOPT_PROXY, proxy.host.c_str());
+    curl_easy_setopt(http_session_.get(), CURLOPT_PROXYPORT, proxy.port);
+    if (!proxy.username.empty()) {
+      curl_easy_setopt(http_session_.get(), CURLOPT_PROXYAUTH, CURLAUTH_ANY);
+      std::string value = proxy.username + ":" + proxy.password;
+      curl_easy_setopt(http_session_.get(), CURLOPT_PROXYUSERPWD, value.c_str());
+    }
+  }
 }
 
-void HTTPClient::appendHeader(const std::string &new_header) {
-  headers_ = curl_slist_append(headers_, new_header.c_str());
+void HTTPClient::setRequestHeader(std::string key, std::optional<std::string> value) {
+  if (value)
+    request_headers_[std::move(key)] = std::move(*value);
+  else
+    request_headers_.erase(key);
 }
 
-void HTTPClient::appendHeader(const std::string &key, const std::string &value) {
-  headers_ = curl_slist_append(headers_, utils::StringUtils::join_pack(key, ": ", value).c_str());
-}
+namespace {
+struct CurlSListFreeAll {
+  void operator()(struct curl_slist* slist) const {
+    curl_slist_free_all(slist);
+  }
+};
+
+std::unique_ptr<struct curl_slist, CurlSListFreeAll> getCurlSList(const std::unordered_map<std::string, std::string>& request_headers) {
+  curl_slist* new_list = nullptr;
+  for (const auto& [header_key, header_value] : request_headers)
+    new_list = curl_slist_append(new_list, utils::StringUtils::join_pack(header_key, ": ", header_value).c_str());
 
-void HTTPClient::setUseChunkedEncoding() {
-  headers_ = curl_slist_append(headers_, "Transfer-Encoding: chunked");
+  return {new_list, {}};
 }
+}  // namespace
+
 
 bool HTTPClient::submit() {
-  if (IsNullOrEmpty(url_))
+  if (url_.empty()) {
+    logger_->log_error("Tried to submit to an empty url");
     return false;
+  }
 
-  int absoluteTimeout = std::max(0, 3 * static_cast<int>(read_timeout_ms_.count()));
+  response_data_.clear();
 
-  curl_easy_setopt(http_session_, CURLOPT_NOSIGNAL, 1);
-  // setting it to 0 will result in the default 300 second timeout
-  curl_easy_setopt(http_session_, CURLOPT_CONNECTTIMEOUT_MS, std::max(0, static_cast<int>(connect_timeout_ms_.count())));
-  curl_easy_setopt(http_session_, CURLOPT_TIMEOUT_MS, absoluteTimeout);
+  curl_easy_setopt(http_session_.get(), CURLOPT_NOSIGNAL, 1);
+  curl_easy_setopt(http_session_.get(), CURLOPT_CONNECTTIMEOUT_MS, connect_timeout_.count());
+  curl_easy_setopt(http_session_.get(), CURLOPT_TIMEOUT_MS, getAbsoluteTimeout().count());
 
-  if (read_timeout_ms_.count() > 0) {
+  if (read_timeout_ > 0ms) {
     progress_.reset();
-    curl_easy_setopt(http_session_, CURLOPT_NOPROGRESS, 0);
-    curl_easy_setopt(http_session_, CURLOPT_XFERINFOFUNCTION, onProgress);
-    curl_easy_setopt(http_session_, CURLOPT_XFERINFODATA, this);
+    curl_easy_setopt(http_session_.get(), CURLOPT_NOPROGRESS, 0);
+    curl_easy_setopt(http_session_.get(), CURLOPT_XFERINFOFUNCTION, onProgress);
+    curl_easy_setopt(http_session_.get(), CURLOPT_XFERINFODATA, this);
   } else {
     // the user explicitly set it to 0
-    curl_easy_setopt(http_session_, CURLOPT_NOPROGRESS, 1);
-  }
-  if (headers_ != nullptr) {
-    headers_ = curl_slist_append(headers_, "Expect:");
-    curl_easy_setopt(http_session_, CURLOPT_HTTPHEADER, headers_);
+    curl_easy_setopt(http_session_.get(), CURLOPT_NOPROGRESS, 1);
   }
 
-  if (username_password_) {
-    curl_easy_setopt(http_session_, CURLOPT_USERNAME, username_password_->username.c_str());
-    curl_easy_setopt(http_session_, CURLOPT_PASSWORD, username_password_->password.c_str());
+  auto headers = getCurlSList(request_headers_);
+  if (headers) {
+    curl_slist_append(headers.get(), "Expect:");
+    curl_easy_setopt(http_session_.get(), CURLOPT_HTTPHEADER, headers.get());
   }
 
-  curl_easy_setopt(http_session_, CURLOPT_URL, url_.c_str());
+  curl_easy_setopt(http_session_.get(), CURLOPT_URL, url_.c_str());
   logger_->log_debug("Submitting to %s", url_);
-  if (callback == nullptr) {
-    content_.ptr = &read_callback_;
-    curl_easy_setopt(http_session_, CURLOPT_WRITEFUNCTION, &utils::HTTPRequestResponse::recieve_write);
-    curl_easy_setopt(http_session_, CURLOPT_WRITEDATA, static_cast<void*>(&content_));
-  }
-  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_.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, keepAlive.count());
-    curl_easy_setopt(http_session_, CURLOPT_TCP_KEEPIDLE, keepIdle.count());
-  } else {
-    logger_->log_debug("Not using keep alive");
-    curl_easy_setopt(http_session_, CURLOPT_TCP_KEEPALIVE, 0L);
+  if (read_callback_ == nullptr) {
+    curl_easy_setopt(http_session_.get(), CURLOPT_WRITEFUNCTION, &utils::HTTPRequestResponse::recieve_write);
+    curl_easy_setopt(http_session_.get(), CURLOPT_WRITEDATA, static_cast<void*>(&content_));
   }
+
+  curl_easy_setopt(http_session_.get(), CURLOPT_HEADERFUNCTION, &utils::HTTPHeaderResponse::receive_headers);
+  curl_easy_setopt(http_session_.get(), CURLOPT_HEADERDATA, static_cast<void*>(&response_data_.header_response));
+
   if (form_ != nullptr) {
-    curl_easy_setopt(http_session_, CURLOPT_MIMEPOST, form_);
-  }
-  res = curl_easy_perform(http_session_);
-  if (callback == nullptr) {
-    read_callback_.close();
-  }
-  long http_code;  // NOLINT long due to libcurl API
-  curl_easy_getinfo(http_session_, CURLINFO_RESPONSE_CODE, &http_code);
-  http_code_ = http_code;
-  curl_easy_getinfo(http_session_, CURLINFO_CONTENT_TYPE, &content_type_str_);
-  if (res == CURLE_OPERATION_TIMEDOUT) {
-    logger_->log_error("HTTP operation timed out, with absolute timeout %dms\n", absoluteTimeout);
-  }
-  if (res != CURLE_OK) {
-    logger_->log_error("curl_easy_perform() failed %s on %s, error code %d\n", curl_easy_strerror(res), url_, res);
+    curl_easy_setopt(http_session_.get(), CURLOPT_MIMEPOST, form_.get());
+  }
+  res_ = curl_easy_perform(http_session_.get());
+  if (read_callback_ == nullptr) {
+    content_.close();
+  }
+  long http_code;  // NOLINT(runtime/int) long due to libcurl API
+  curl_easy_getinfo(http_session_.get(), CURLINFO_RESPONSE_CODE, &http_code);
+  response_data_.response_code = http_code;
+  curl_easy_getinfo(http_session_.get(), CURLINFO_CONTENT_TYPE, &response_data_.response_content_type);
+  if (res_ == CURLE_OPERATION_TIMEDOUT) {
+    logger_->log_error("HTTP operation timed out, with absolute timeout %" PRId64 "ms\n", getAbsoluteTimeout().count());
+  }
+  if (res_ != CURLE_OK) {
+    logger_->log_info("%d", request_headers_.size());
+    logger_->log_error("curl_easy_perform() failed %s on %s, error code %d\n", curl_easy_strerror(res_), url_, res_);
     return false;
   }
 
   logger_->log_debug("Finished with %s", url_);
   return true;
 }
 
-CURLcode HTTPClient::getResponseResult() {
-  return res;
-}
-
 int64_t HTTPClient::getResponseCode() const {
-  return http_code_;
+  return response_data_.response_code;
 }
 
 const char *HTTPClient::getContentType() {
-  return content_type_str_;
+  return response_data_.response_content_type;
 }
 
 const std::vector<char> &HTTPClient::getResponseBody() {
-  if (response_body_.empty()) {
-    if (callback && callback->ptr) {
-      response_body_ = callback->ptr->to_string();
+  if (response_data_.response_body.empty()) {
+    if (read_callback_) {
+      response_data_.response_body = read_callback_->to_string();
     } else {
-      response_body_ = read_callback_.to_string();
+      response_data_.response_body = content_.to_string();
     }
   }
-  return response_body_;
-}
-
-void HTTPClient::set_request_method(const std::string method) {
-  std::string my_method = method;
-  std::transform(my_method.begin(), my_method.end(), my_method.begin(), ::toupper);
-  if (my_method == "POST") {
-    curl_easy_setopt(http_session_, CURLOPT_POST, 1L);
-  } else if (my_method == "PUT") {
-    curl_easy_setopt(http_session_, CURLOPT_UPLOAD, 1L);
-  } else if (my_method == "HEAD") {
-    curl_easy_setopt(http_session_, CURLOPT_NOBODY, 1L);
-  } else if (my_method == "GET") {
+  return response_data_.response_body;
+}
+
+void HTTPClient::set_request_method(std::string method) {
+  if (method_ == method)
+    return;
+  method_ = std::move(method);
+  std::transform(method_.begin(), method_.end(), method_.begin(), ::toupper);

Review Comment:
   two nitpicking comments:
   * if we call `set_request_method("put")` 100 times, then `curl_easy_setopt` will be called 200 times;
   * `toupper(negative number)` is UB.
   
   Also, we have `range v3`, so I would write it like this:
   ```suggestion
     ranges::actions::transform(method, [](auto ch) { return ::toupper(static_cast<unsigned char>(ch)); });
     if (method_ == method)
       return;
     method_ = std::move(method);
   ```



-- 
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