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/09 11:25:20 UTC

[GitHub] [nifi-minifi-cpp] martinzink opened a new pull request, #1383: MINIFICPP-1875 HTTPClient should be reusable

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

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


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

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

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


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

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


##########
libminifi/test/unit/ResourceQueueTests.cpp:
##########
@@ -0,0 +1,90 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include <chrono>
+#include <set>
+
+#include "ResourceQueue.h"
+#include "../TestBase.h"
+#include "../Catch.h"
+#include "logging/LoggerConfiguration.h"
+
+using namespace std::literals::chrono_literals;
+
+namespace org::apache::nifi::minifi::utils::testing {
+
+TEST_CASE("maximum_number_of_creatable_resources", "[utils::ResourceQueue]") {
+  std::shared_ptr<core::logging::Logger> logger_{core::logging::LoggerFactory<ResourceQueue<int>>::getLogger()};
+  LogTestController::getInstance().setTrace<ResourceQueue<int>>();
+
+  std::set<int> resources_created;
+
+  auto worker = [&](int value, const std::shared_ptr<ResourceQueue<int>>& resource_queue) {
+    auto resource = resource_queue->getResource([value]{return std::make_unique<int>(value);});
+    std::this_thread::sleep_for(10ms);
+    resources_created.emplace(*resource);
+  };
+
+  SECTION("Maximum 2 resources") {
+    auto resource_queue = ResourceQueue<int>::create(2, logger_);
+    std::thread thread_one{[&] { worker(1, resource_queue); }};
+    std::thread thread_two{[&] { worker(2, resource_queue); }};
+    std::thread thread_three{[&] { worker(3, resource_queue); }};
+
+    thread_one.join();
+    thread_two.join();
+    thread_three.join();
+
+    CHECK(!resources_created.empty());
+    CHECK(resources_created.size() <= 2);
+  }
+
+
+  SECTION("No Maximum resources") {
+    auto resource_queue = ResourceQueue<int>::create(2, logger_);

Review Comment:
   nice catch https://github.com/apache/nifi-minifi-cpp/pull/1383/commits/3f4aeec40af42fb9df1b15fbf0c2298664e77675#diff-18dc99071b15b546122f8e94bbfd19397b0e4c5d1d9d1d385234b19c84881c76R59



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

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

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


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

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


##########
libminifi/test/unit/ResourceQueueTests.cpp:
##########
@@ -0,0 +1,90 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include <chrono>
+#include <set>
+
+#include "ResourceQueue.h"
+#include "../TestBase.h"
+#include "../Catch.h"
+#include "logging/LoggerConfiguration.h"
+
+using namespace std::literals::chrono_literals;
+
+namespace org::apache::nifi::minifi::utils::testing {
+
+TEST_CASE("maximum_number_of_creatable_resources", "[utils::ResourceQueue]") {
+  std::shared_ptr<core::logging::Logger> logger_{core::logging::LoggerFactory<ResourceQueue<int>>::getLogger()};
+  LogTestController::getInstance().setTrace<ResourceQueue<int>>();
+
+  std::set<int> resources_created;
+
+  auto worker = [&](int value, const std::shared_ptr<ResourceQueue<int>>& resource_queue) {
+    auto resource = resource_queue->getResource([value]{return std::make_unique<int>(value);});
+    std::this_thread::sleep_for(10ms);
+    resources_created.emplace(*resource);
+  };
+
+  SECTION("Maximum 2 resources") {
+    auto resource_queue = ResourceQueue<int>::create(2, logger_);
+    std::thread thread_one{[&] { worker(1, resource_queue); }};
+    std::thread thread_two{[&] { worker(2, resource_queue); }};
+    std::thread thread_three{[&] { worker(3, resource_queue); }};
+
+    thread_one.join();
+    thread_two.join();
+    thread_three.join();
+
+    CHECK(!resources_created.empty());
+    CHECK(resources_created.size() <= 2);
+  }
+
+
+  SECTION("No Maximum resources") {
+    auto resource_queue = ResourceQueue<int>::create(2, logger_);
+    std::thread thread_one{[&] { worker(1, resource_queue); }};
+    std::thread thread_two{[&] { worker(2, resource_queue); }};
+    std::thread thread_three{[&] { worker(3, resource_queue); }};
+
+    thread_one.join();
+    thread_two.join();
+    thread_three.join();
+
+    CHECK(!resources_created.empty());
+    CHECK(resources_created.size() <= 3);

Review Comment:
   you need to reset the logger, as the other section prints "Waiting for resource"
   
   or, probably better, you could look for "Number of instances: 3"



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

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

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


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

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


##########
extensions/http-curl/processors/InvokeHTTP.h:
##########
@@ -127,52 +130,37 @@ class InvokeHTTP : public core::Processor {
 
   EXTENSIONAPI static constexpr const char* STATUS_CODE = "invokehttp.status.code";
   EXTENSIONAPI static constexpr const char* STATUS_MESSAGE = "invokehttp.status.message";
-  EXTENSIONAPI static constexpr const char* RESPONSE_BODY = "invokehttp.response.body";
   EXTENSIONAPI static constexpr const char* REQUEST_URL = "invokehttp.request.url";
   EXTENSIONAPI static constexpr const char* TRANSACTION_ID = "invokehttp.tx.id";
-  EXTENSIONAPI static constexpr const char* REMOTE_DN = "invokehttp.remote.dn";
-  EXTENSIONAPI static constexpr const char* EXCEPTION_CLASS = "invokehttp.java.exception.class";
-  EXTENSIONAPI static constexpr const char* EXCEPTION_MESSAGE = "invokehttp.java.exception.message";
 
-  void onTrigger(const std::shared_ptr<core::ProcessContext> &context, const std::shared_ptr<core::ProcessSession> &session) override;
+  void onTrigger(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSession>& session) override;
   void initialize() override;
-  void onSchedule(const std::shared_ptr<core::ProcessContext> &context, const std::shared_ptr<core::ProcessSessionFactory> &sessionFactory) override;
+  void onSchedule(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSessionFactory>& sessionFactory) override;
 
  private:
-  /**
-   * Routes the flowfile to the proper destination
-   * @param request request flow file record
-   * @param response response flow file record
-   * @param session process session
-   * @param context process context
-   * @param isSuccess success code or not
-   * @param statuscode http response code.
-   */
-  void route(const std::shared_ptr<core::FlowFile> &request, const std::shared_ptr<core::FlowFile> &response, const std::shared_ptr<core::ProcessSession> &session,
-             const std::shared_ptr<core::ProcessContext> &context, bool is_success, int64_t status_code);
-  bool shouldEmitFlowFile() const;
+  void route(const std::shared_ptr<core::FlowFile>& request, const std::shared_ptr<core::FlowFile>& response, const std::shared_ptr<core::ProcessSession>& session,
+             const std::shared_ptr<core::ProcessContext>& context, bool is_success, int64_t status_code);
+  static bool shouldEmitFlowFile(minifi::extensions::curl::HTTPClient& client);
+  void onTriggerWithClient(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSession>& session, minifi::extensions::curl::HTTPClient& client);
   [[nodiscard]] bool appendHeaders(const core::FlowFile& flow_file, /*std::invocable<std::string, std::string>*/ auto append_header);
 
-  std::shared_ptr<minifi::controllers::SSLContextService> ssl_context_service_;
-  std::string method_;
-  std::string url_;
-  bool date_header_include_{true};
+
+  void setupMembersFromProperties(const core::ProcessContext& context);
+  std::unique_ptr<minifi::extensions::curl::HTTPClient> createHTTPClientFromPropertiesAndMembers(const core::ProcessContext& context) const;
+
   std::optional<utils::Regex> attributes_to_send_;
-  std::chrono::milliseconds connect_timeout_ms_{20000};
-  std::chrono::milliseconds read_timeout_ms_{20000};
-  // attribute in which response body will be added
-  std::string put_attribute_name_;
+
+  std::optional<std::string> put_response_body_in_attribute_;
   bool always_output_response_{false};
-  std::string content_type_;
   bool use_chunked_encoding_{false};
   bool penalize_no_retry_{false};
-  // disabling peer verification makes susceptible for MITM attacks
-  bool disable_peer_verification_{false};
-  utils::HTTPProxy proxy_;
-  bool follow_redirects_{true};
   bool send_body_{true};
+
   InvalidHTTPHeaderFieldHandlingOption invalid_http_header_field_handling_strategy_;
+
   std::shared_ptr<core::logging::Logger> logger_{core::logging::LoggerFactory<InvokeHTTP>::getLogger()};
+  gsl::not_null<std::shared_ptr<utils::ResourceQueue<extensions::curl::HTTPClient>>> client_queue_ = gsl::make_not_null(
+      utils::ResourceQueue<extensions::curl::HTTPClient>::create(getMaxConcurrentTasks(), logger_));

Review Comment:
   A dummy object with incorrect data is more dangerous, than null. I agree.



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

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

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


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

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


##########
extensions/http-curl/client/HTTPClient.cpp:
##########
@@ -378,10 +354,11 @@ int HTTPClient::onProgress(void *clientp, curl_off_t /*dltotal*/, curl_off_t dln
     return 0;
   }
   // did not transfer data
-  if (elapsed.count() > client.read_timeout_ms_.count()) {
+  if (elapsed > client.read_timeout_) {
     // timeout
-    client.logger_->log_error("HTTP operation has been idle for %dms, limit (%dms) reached, terminating connection\n",
-      static_cast<int>(elapsed.count()), static_cast<int>(client.read_timeout_ms_.count()));
+    client.logger_->log_error("HTTP operation has been idle for %" PRId64 " ms, limit (%" PRId64 "ms) reached, terminating connection\n",
+                              std::chrono::duration_cast<std::chrono::milliseconds>(elapsed).count(),
+                              client.read_timeout_.count());

Review Comment:
   good idea :+1: https://github.com/apache/nifi-minifi-cpp/pull/1383/commits/3edf5fcfd603463b67bd6c072fa0128bbb861787#diff-e83ea756e38995b3194ecc99d0a64fc19c8ef992fb7a575a34def27a67f7819aR380-R381



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

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

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


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

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


##########
libminifi/include/utils/ResourceQueue.h:
##########
@@ -0,0 +1,125 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#pragma once
+
+#include <cstdint>
+#include <memory>
+#include <optional>
+#include <string>
+#include <utility>
+#include <functional>
+#include <mutex>
+#include <condition_variable>
+
+#include "core/logging/Logger.h"
+#include "concurrentqueue.h"
+#include "MinifiConcurrentQueue.h"
+
+namespace org::apache::nifi::minifi::utils {
+
+template<class ResourceType>
+class ResourceQueue : public std::enable_shared_from_this<ResourceQueue<ResourceType>> {

Review Comment:
   good idea, I've added some explanatory comments in https://github.com/apache/nifi-minifi-cpp/commit/3edf5fcfd603463b67bd6c072fa0128bbb861787#diff-4205c4389dbd9a8dee16d4d38da02ada57c19d038893abdcb3f29424c7d1dd86R36-R42



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

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

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


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

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


##########
libminifi/include/utils/BaseHTTPClient.h:
##########
@@ -51,38 +39,30 @@ 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;
+class HTTPUploadCallback : public ByteInputCallback {
+ public:
+  template<typename... Args>
+  explicit HTTPUploadCallback(Args&& ... args) : ByteInputCallback(std::forward<Args>(args)...) {}
 
   size_t getPos() {
-    std::lock_guard<std::mutex> lock(mutex);
     return pos;
   }

Review Comment:
   After removing the lock is this strictly needed anymore? The position member is public and could be used as it is. Same for HTTPReadCallback



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

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

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


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

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


##########
extensions/http-curl/processors/InvokeHTTP.h:
##########
@@ -127,52 +130,37 @@ class InvokeHTTP : public core::Processor {
 
   EXTENSIONAPI static constexpr const char* STATUS_CODE = "invokehttp.status.code";
   EXTENSIONAPI static constexpr const char* STATUS_MESSAGE = "invokehttp.status.message";
-  EXTENSIONAPI static constexpr const char* RESPONSE_BODY = "invokehttp.response.body";
   EXTENSIONAPI static constexpr const char* REQUEST_URL = "invokehttp.request.url";
   EXTENSIONAPI static constexpr const char* TRANSACTION_ID = "invokehttp.tx.id";
-  EXTENSIONAPI static constexpr const char* REMOTE_DN = "invokehttp.remote.dn";
-  EXTENSIONAPI static constexpr const char* EXCEPTION_CLASS = "invokehttp.java.exception.class";
-  EXTENSIONAPI static constexpr const char* EXCEPTION_MESSAGE = "invokehttp.java.exception.message";
 
-  void onTrigger(const std::shared_ptr<core::ProcessContext> &context, const std::shared_ptr<core::ProcessSession> &session) override;
+  void onTrigger(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSession>& session) override;
   void initialize() override;
-  void onSchedule(const std::shared_ptr<core::ProcessContext> &context, const std::shared_ptr<core::ProcessSessionFactory> &sessionFactory) override;
+  void onSchedule(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSessionFactory>& sessionFactory) override;
 
  private:
-  /**
-   * Routes the flowfile to the proper destination
-   * @param request request flow file record
-   * @param response response flow file record
-   * @param session process session
-   * @param context process context
-   * @param isSuccess success code or not
-   * @param statuscode http response code.
-   */
-  void route(const std::shared_ptr<core::FlowFile> &request, const std::shared_ptr<core::FlowFile> &response, const std::shared_ptr<core::ProcessSession> &session,
-             const std::shared_ptr<core::ProcessContext> &context, bool is_success, int64_t status_code);
-  bool shouldEmitFlowFile() const;
+  void route(const std::shared_ptr<core::FlowFile>& request, const std::shared_ptr<core::FlowFile>& response, const std::shared_ptr<core::ProcessSession>& session,
+             const std::shared_ptr<core::ProcessContext>& context, bool is_success, int64_t status_code);
+  static bool shouldEmitFlowFile(minifi::extensions::curl::HTTPClient& client);
+  void onTriggerWithClient(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSession>& session, minifi::extensions::curl::HTTPClient& client);
   [[nodiscard]] bool appendHeaders(const core::FlowFile& flow_file, /*std::invocable<std::string, std::string>*/ auto append_header);
 
-  std::shared_ptr<minifi::controllers::SSLContextService> ssl_context_service_;
-  std::string method_;
-  std::string url_;
-  bool date_header_include_{true};
+
+  void setupMembersFromProperties(const core::ProcessContext& context);
+  std::unique_ptr<minifi::extensions::curl::HTTPClient> createHTTPClientFromPropertiesAndMembers(const core::ProcessContext& context) const;
+
   std::optional<utils::Regex> attributes_to_send_;
-  std::chrono::milliseconds connect_timeout_ms_{20000};
-  std::chrono::milliseconds read_timeout_ms_{20000};
-  // attribute in which response body will be added
-  std::string put_attribute_name_;
+
+  std::optional<std::string> put_response_body_in_attribute_;
   bool always_output_response_{false};
-  std::string content_type_;
   bool use_chunked_encoding_{false};
   bool penalize_no_retry_{false};
-  // disabling peer verification makes susceptible for MITM attacks
-  bool disable_peer_verification_{false};
-  utils::HTTPProxy proxy_;
-  bool follow_redirects_{true};
   bool send_body_{true};
+
   InvalidHTTPHeaderFieldHandlingOption invalid_http_header_field_handling_strategy_;
+
   std::shared_ptr<core::logging::Logger> logger_{core::logging::LoggerFactory<InvokeHTTP>::getLogger()};
+  gsl::not_null<std::shared_ptr<utils::ResourceQueue<extensions::curl::HTTPClient>>> client_queue_ = gsl::make_not_null(
+      utils::ResourceQueue<extensions::curl::HTTPClient>::create(getMaxConcurrentTasks(), logger_));

Review Comment:
   I am not sure, but it will be overwritten by onSchedule anyways



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

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

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


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

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


##########
extensions/http-curl/tests/unit/ConnectionCountingServer.h:
##########
@@ -0,0 +1,149 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#pragma once
+
+#include <array>
+#include <vector>
+#include <string>
+#include <set>
+#include "CivetServer.h"
+
+namespace org::apache::nifi::minifi::extensions::curl::testing {
+
+namespace details {
+
+class NumberedMethodResponder : public CivetHandler {
+ public:
+  explicit NumberedMethodResponder(std::set<utils::SmallString<36>>& connections) : connections_(connections) {}
+
+  bool handleGet(CivetServer*, struct mg_connection* conn) override {
+    sendNumberedMessage("GET", conn);
+    return true;
+  }
+
+  bool handlePost(CivetServer*, struct mg_connection* conn) override {
+    sendNumberedMessage("POST", conn);
+    return true;
+  }
+
+  bool handlePut(CivetServer*, struct mg_connection* conn) override {
+    sendNumberedMessage("PUT", conn);
+    return true;
+  }
+
+  bool handleHead(CivetServer*, struct mg_connection* conn) override {
+    sendNumberedMessage("HEAD", conn);
+    return true;
+  }
+
+ private:
+  void sendNumberedMessage(std::string body, struct mg_connection* conn) {
+    saveConnectionId(conn);
+    body.append(std::to_string(response_id_));
+    mg_printf(conn, "HTTP/1.1 200 OK\r\n");
+    mg_printf(conn, "Content-length: %lu\r\n", body.length());
+    mg_printf(conn, "Response-number: %" PRIu64 "\r\n", response_id_);
+    mg_printf(conn, "\r\n");
+    mg_printf(conn, body.data(), body.length());
+    ++response_id_;
+  }
+
+  void saveConnectionId(struct mg_connection* conn) {
+    auto user_connection_data = reinterpret_cast<utils::SmallString<36>*>(mg_get_user_connection_data(conn));
+    connections_.emplace(*user_connection_data);
+  }
+
+  uint64_t response_id_ = 0;
+  std::set<utils::SmallString<36>>& connections_;
+};
+
+class ReverseBodyPostHandler : public CivetHandler {
+ public:
+  explicit ReverseBodyPostHandler(std::set<utils::SmallString<36>>& connections) : connections_(connections) {}
+
+  bool handlePost(CivetServer* /*server*/, struct mg_connection* conn) override {
+    saveConnectionId(conn);
+    std::vector<char> request_body;
+    request_body.reserve(2048);
+    size_t read_size = mg_read(conn, request_body.data(), 2048);

Review Comment:
   whoosh... fixed it in https://github.com/apache/nifi-minifi-cpp/pull/1383/commits/b85658fa591dea6f93cb8877f1c38c34039a0a19 thanks



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

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

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


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

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


##########
libminifi/test/unit/ResourceQueueTests.cpp:
##########
@@ -0,0 +1,90 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include <chrono>
+#include <set>
+
+#include "ResourceQueue.h"
+#include "../TestBase.h"
+#include "../Catch.h"
+#include "logging/LoggerConfiguration.h"
+
+using namespace std::literals::chrono_literals;
+
+namespace org::apache::nifi::minifi::utils::testing {
+
+TEST_CASE("maximum_number_of_creatable_resources", "[utils::ResourceQueue]") {
+  std::shared_ptr<core::logging::Logger> logger_{core::logging::LoggerFactory<ResourceQueue<int>>::getLogger()};
+  LogTestController::getInstance().setTrace<ResourceQueue<int>>();
+
+  std::set<int> resources_created;
+
+  auto worker = [&](int value, const std::shared_ptr<ResourceQueue<int>>& resource_queue) {
+    auto resource = resource_queue->getResource([value]{return std::make_unique<int>(value);});
+    std::this_thread::sleep_for(10ms);
+    resources_created.emplace(*resource);
+  };
+
+  SECTION("Maximum 2 resources") {
+    auto resource_queue = ResourceQueue<int>::create(2, logger_);
+    std::thread thread_one{[&] { worker(1, resource_queue); }};
+    std::thread thread_two{[&] { worker(2, resource_queue); }};
+    std::thread thread_three{[&] { worker(3, resource_queue); }};
+
+    thread_one.join();
+    thread_two.join();
+    thread_three.join();
+
+    CHECK(!resources_created.empty());
+    CHECK(resources_created.size() <= 2);
+  }
+
+
+  SECTION("No Maximum resources") {
+    auto resource_queue = ResourceQueue<int>::create(2, logger_);
+    std::thread thread_one{[&] { worker(1, resource_queue); }};
+    std::thread thread_two{[&] { worker(2, resource_queue); }};
+    std::thread thread_three{[&] { worker(3, resource_queue); }};
+
+    thread_one.join();
+    thread_two.join();
+    thread_three.join();
+
+    CHECK(!resources_created.empty());
+    CHECK(resources_created.size() <= 3);

Review Comment:
   you need to reset the logger, as the other section prints "Waiting for resource"



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

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

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


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

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


##########
libminifi/include/utils/ResourceQueue.h:
##########
@@ -0,0 +1,132 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#pragma once
+
+#include <cstdint>
+#include <memory>
+#include <optional>
+#include <string>
+#include <utility>
+#include <functional>
+#include <mutex>
+#include <condition_variable>
+
+#include "core/logging/Logger.h"
+#include "concurrentqueue.h"
+#include "MinifiConcurrentQueue.h"
+
+namespace org::apache::nifi::minifi::utils {
+
+/*
+ * utils::ResourceQueue a threadsafe resource pool that lends out existing resources or creates them if necessary.
+ * getResource will return an existing unused resource or use the create_resource function to create one.
+ * If the number of existing resources reached the maximum_number_of_creatable_resources_, the call will block until a resource is returned to the queue.
+ * The lent out resource is in a ResourceWrapper that returns the resource to the queue on destruction.
+ * */
+
+template<class ResourceType>
+class ResourceQueue : public std::enable_shared_from_this<ResourceQueue<ResourceType>> {
+ public:
+  class ResourceWrapper {
+   public:
+    ResourceWrapper(std::weak_ptr<ResourceQueue<ResourceType>> queue, std::unique_ptr<ResourceType>&& resource) : queue_(std::move(queue)), resource_(std::move(resource)) {}
+    ResourceWrapper(ResourceWrapper&& src) = default;
+    ResourceWrapper(const ResourceWrapper&) = delete;
+    ~ResourceWrapper() {
+      if (auto queue = queue_.lock())
+        queue->returnResource(std::move(resource_));
+    }
+
+    ResourceWrapper& operator=(ResourceWrapper&&) = default;
+    ResourceWrapper& operator=(const ResourceWrapper&) = delete;
+
+    ResourceType& operator*() const { return *resource_; }
+    ResourceType* operator->() const noexcept { return resource_.operator->(); }
+    ResourceType* get() const { return resource_.get(); }
+
+   private:
+    std::weak_ptr<ResourceQueue<ResourceType>> queue_;
+    std::unique_ptr<ResourceType> resource_;
+  };
+
+  static auto create(std::optional<size_t> maximum_number_of_creatable_resources, std::shared_ptr<core::logging::Logger> logger);
+
+  template<typename Fn>
+  [[nodiscard]] std::enable_if_t<std::is_invocable_v<std::unique_ptr<ResourceType>()>, ResourceWrapper> getResource(const Fn& create_resource) {
+    std::unique_ptr<ResourceType> resource;
+    // Use an existing resource, if one is available
+    if (internal_queue_.tryDequeue(resource)) {
+      logDebug("Using available [%p] resource instance", resource.get());
+      return ResourceWrapper(this->weak_from_this(), std::move(resource));
+    } else {
+      const std::lock_guard<std::mutex> lock(counter_mutex_);
+      if (!maximum_number_of_creatable_resources_ || resources_created_ < maximum_number_of_creatable_resources_) {
+        ++resources_created_;
+        resource = create_resource();
+        logDebug("Created new [%p] resource instance. Number of instances: %d%s.",
+                 resource.get(),
+                 resources_created_,
+                 maximum_number_of_creatable_resources_ ? " / " + std::to_string(*maximum_number_of_creatable_resources_) : "");
+        return ResourceWrapper(this->weak_from_this(), std::move(resource));
+      }
+    }
+    logDebug("Waiting for resource");
+    if (!internal_queue_.dequeueWait(resource)) {
+      throw std::runtime_error("No resource available");
+    }
+    return ResourceWrapper(this->weak_from_this(), std::move(resource));
+  }
+
+ protected:
+  ResourceQueue(std::optional<size_t> maximum_number_of_creatable_resources, std::shared_ptr<core::logging::Logger> logger)
+      : maximum_number_of_creatable_resources_(maximum_number_of_creatable_resources),
+        logger_(std::move(logger)) {
+  }
+
+ private:
+  void returnResource(std::unique_ptr<ResourceType>&& resource) {

Review Comment:
   nope, I've removed it in https://github.com/apache/nifi-minifi-cpp/pull/1383/commits/7da01d53dfe1e127c2173fcbc9b9a9fadc712ff3



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

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

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


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

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


##########
extensions/http-curl/tests/unit/ConnectionCountingServer.h:
##########
@@ -0,0 +1,149 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#pragma once
+
+#include <array>
+#include <vector>
+#include <string>
+#include <set>
+#include "CivetServer.h"
+
+namespace org::apache::nifi::minifi::extensions::curl::testing {
+
+namespace details {
+
+class NumberedMethodResponder : public CivetHandler {
+ public:
+  explicit NumberedMethodResponder(std::set<utils::SmallString<36>>& connections) : connections_(connections) {}
+
+  bool handleGet(CivetServer*, struct mg_connection* conn) override {
+    sendNumberedMessage("GET", conn);
+    return true;
+  }
+
+  bool handlePost(CivetServer*, struct mg_connection* conn) override {
+    sendNumberedMessage("POST", conn);
+    return true;
+  }
+
+  bool handlePut(CivetServer*, struct mg_connection* conn) override {
+    sendNumberedMessage("PUT", conn);
+    return true;
+  }
+
+  bool handleHead(CivetServer*, struct mg_connection* conn) override {
+    sendNumberedMessage("HEAD", conn);
+    return true;
+  }
+
+ private:
+  void sendNumberedMessage(std::string body, struct mg_connection* conn) {
+    saveConnectionId(conn);
+    body.append(std::to_string(response_id_));
+    mg_printf(conn, "HTTP/1.1 200 OK\r\n");
+    mg_printf(conn, "Content-length: %lu\r\n", body.length());
+    mg_printf(conn, "Response-number: %" PRIu64 "\r\n", response_id_);
+    mg_printf(conn, "\r\n");
+    mg_printf(conn, body.data(), body.length());
+    ++response_id_;
+  }
+
+  void saveConnectionId(struct mg_connection* conn) {
+    auto user_connection_data = reinterpret_cast<utils::SmallString<36>*>(mg_get_user_connection_data(conn));
+    connections_.emplace(*user_connection_data);
+  }
+
+  uint64_t response_id_ = 0;
+  std::set<utils::SmallString<36>>& connections_;
+};
+
+class ReverseBodyPostHandler : public CivetHandler {
+ public:
+  explicit ReverseBodyPostHandler(std::set<utils::SmallString<36>>& connections) : connections_(connections) {}
+
+  bool handlePost(CivetServer* /*server*/, struct mg_connection* conn) override {
+    saveConnectionId(conn);
+    std::vector<char> request_body;
+    request_body.reserve(2048);
+    size_t read_size = mg_read(conn, request_body.data(), 2048);
+    std::string response_body{request_body.begin(), request_body.begin() + read_size};
+    std::reverse(std::begin(response_body), std::end(response_body));
+    mg_printf(conn, "HTTP/1.1 200 OK\r\n");
+    mg_printf(conn, "Content-length: %zu\r\n", read_size);
+    mg_printf(conn, "\r\n");
+    mg_printf(conn, response_body.data(), read_size);
+
+    return true;
+  }
+
+ private:
+  void saveConnectionId(struct mg_connection* conn) {
+    auto user_connection_data = reinterpret_cast<utils::SmallString<36>*>(mg_get_user_connection_data(conn));
+    connections_.emplace(*user_connection_data);
+  }
+
+  std::set<utils::SmallString<36>>& connections_;
+};
+
+struct AddIdToUserConnectionData : public CivetCallbacks {
+  AddIdToUserConnectionData() {
+    init_connection = [](const struct mg_connection*, void** user_connection_data) -> int {
+      utils::SmallString<36>* id = new utils::SmallString<36>(utils::IdGenerator::getIdGenerator()->generate().to_string());
+      *user_connection_data = reinterpret_cast<void*>(id);
+      return 0;
+    };
+
+    connection_close = [](const struct mg_connection* conn) -> void {
+      auto user_connection_data = reinterpret_cast<utils::SmallString<36>*>(mg_get_user_connection_data(conn));
+      delete user_connection_data;
+    };

Review Comment:
   I see, thanks for clarifying!



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

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

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


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

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


##########
docker/test/integration/steps/steps.py:
##########
@@ -271,6 +271,12 @@ def step_impl(context, content, path):
     context.test.add_test_data(path, content)
 
 
+@given("{number_of_files:d} files with the content \"{content}\" are present in \"{path}\"")
+def step_impl(context, number_of_files, content, path):

Review Comment:
   I would delegate the old single file version to this to avoid repetition



##########
extensions/http-curl/client/HTTPClient.cpp:
##########
@@ -32,62 +31,53 @@
 #include "range/v3/algorithm/all_of.hpp"
 #include "range/v3/action/transform.hpp"
 
-namespace org::apache::nifi::minifi::utils {
+using namespace std::literals::chrono_literals;
+
+namespace org::apache::nifi::minifi::extensions::curl {
 
-HTTPClient::HTTPClient(std::string url, const std::shared_ptr<minifi::controllers::SSLContextService> ssl_context_service)
+HTTPClient::HTTPClient(std::string url, std::shared_ptr<minifi::controllers::SSLContextService>  ssl_context_service)
     : core::Connectable("HTTPClient"),
-      ssl_context_service_(ssl_context_service),
+      ssl_context_service_(std::move(ssl_context_service)),
       url_(std::move(url)) {
-  http_session_ = curl_easy_init();
+  http_session_.reset(curl_easy_init());
 }
 
 HTTPClient::HTTPClient(const std::string& name, const utils::Identifier& uuid)
     : core::Connectable(name, uuid) {
-  http_session_ = curl_easy_init();
+  http_session_.reset(curl_easy_init());
 }
 
 HTTPClient::HTTPClient()
     : core::Connectable("HTTPClient") {
-  http_session_ = curl_easy_init();
+  http_session_.reset(curl_easy_init());
 }
 
-void HTTPClient::addFormPart(const std::string& content_type, const std::string& name, HTTPUploadCallback* read_callback, const std::optional<std::string>& filename) {
+void HTTPClient::addFormPart(const std::string& content_type, const std::string& name, std::unique_ptr<utils::HTTPUploadCallback>&& form_callback, const std::optional<std::string>& filename) {

Review Comment:
   I would pass the unique_ptr by value, just so that it's clear from the interface that the passed-in object IS being moved from as opposed to "may be moved from". It's a more specific/accurate lifetime specification on the interface, although it may (but [actually doesn't](https://godbolt.org/z/Y8nMe9WhM)) result in one extra pointer copy.



##########
extensions/http-curl/client/HTTPClient.cpp:
##########
@@ -164,212 +161,191 @@ 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::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) {

Review Comment:
   same here: I would just make it absolutely clear that this is gone after the call.
   ```suggestion
   void HTTPClient::setReadCallback(std::unique_ptr<utils::HTTPReadCallback> callback) {
   ```



##########
extensions/http-curl/processors/InvokeHTTP.cpp:
##########
@@ -46,31 +45,31 @@ namespace org::apache::nifi::minifi::processors {
 std::string InvokeHTTP::DefaultContentType = "application/octet-stream";
 
 const core::Property InvokeHTTP::Method("HTTP Method", "HTTP request method (GET, POST, PUT, PATCH, DELETE, HEAD, OPTIONS). "
-                                  "Arbitrary methods are also supported. Methods other than POST, PUT and PATCH will be sent without a message body.",
-                                  "GET");
+                                                       "Arbitrary methods are also supported. Methods other than POST, PUT and PATCH will be sent without a message body.",
+                                        "GET");

Review Comment:
   I don't like alignment in continuation in general, but this is even misaligned now. If you choose alignment, it should be aligned to the opening parenthesis.



##########
extensions/http-curl/client/RequestHeaders.h:
##########
@@ -0,0 +1,58 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#pragma once
+
+#include <unordered_map>
+#include <memory>
+#include <optional>
+#include <string>
+#include <string_view>
+
+#ifdef WIN32
+#pragma comment(lib, "wldap32.lib" )
+#pragma comment(lib, "crypt32.lib" )
+#pragma comment(lib, "Ws2_32.lib")
+
+#define CURL_STATICLIB
+#include <curl/curl.h>
+#else
+#include <curl/curl.h>
+#endif
+
+namespace org::apache::nifi::minifi::extensions::curl {
+class RequestHeaders {
+ public:
+  RequestHeaders() = default;
+
+  void appendHeader(std::string key, std::string value);
+
+  void disableExpectHeader();
+
+  [[nodiscard]] std::unique_ptr<struct curl_slist, decltype(&curl_slist_free_all)> get() const;
+  [[nodiscard]] auto size() const { return headers_.size(); }
+  [[nodiscard]] bool empty() const;
+
+  std::string& operator[](const std::string& key);
+  std::string& operator[](std::string&& key);
+
+  [[nodiscard]] bool contains(const std::string& key) const;
+  void erase(const std::string& key);
+
+ private:
+  std::unordered_map<std::string, std::string> headers_;
+};

Review Comment:
   Is there any reason to introduce this class instead of just using `std::unordered_map<std::string, std::string>` directly (and a function to create `curl_slist` from it)?



##########
extensions/http-curl/client/HTTPCallback.h:
##########
@@ -156,11 +152,11 @@ class HttpStreamingCallback final : public ByteInputCallback {
       current_pos_ = current_buffer_start_;
       total_bytes_loaded_ += current_vec_.size();
       logger_->log_trace("loadNextBuffer() loaded new buffer, ptr_: %p, size: %zu, current_buffer_start_: %zu, current_pos_: %zu, total_bytes_loaded_: %zu",
-          ptr_,
-          current_vec_.size(),
-          current_buffer_start_,
-          current_pos_,
-          total_bytes_loaded_);
+                         ptr_,
+                         current_vec_.size(),
+                         current_buffer_start_,
+                         current_pos_,
+                         total_bytes_loaded_);

Review Comment:
   I prefer 2 levels as continuation indentation instead of alignment, but no strong feelings.



##########
extensions/http-curl/client/HTTPClient.h:
##########
@@ -51,17 +53,28 @@
 #include "core/logging/Logger.h"
 #include "core/logging/LoggerConfiguration.h"
 
-namespace org::apache::nifi::minifi::utils {
+namespace org::apache::nifi::minifi::extensions::curl {
 
-/**
- * Purpose and Justification: Pull the basics for an HTTPClient into a self contained class. Simply provide
- * the URL and an SSLContextService ( can be null).
- *
- * Since several portions of the code have been relying on curl, we can encapsulate most CURL HTTP
- * operations here without maintaining it everywhere. Further, this will help with testing as we
- * only need to to test our usage of CURL once
- */
-class HTTPClient : public BaseHTTPClient, public core::Connectable {
+struct KeepAliveProbeData {
+  std::chrono::seconds keep_alive_delay;
+  std::chrono::seconds keep_alive_interval;
+};
+
+struct HTTPResponseData {
+  std::vector<char> response_body;
+  utils::HTTPHeaderResponse header_response;
+  char* response_content_type;
+  int64_t response_code;

Review Comment:
   Would you mind adding default member initializers to the primitive types, just to make sure that nothing is uninitialized?



##########
libminifi/include/utils/ResourceQueue.h:
##########
@@ -0,0 +1,125 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#pragma once
+
+#include <cstdint>
+#include <memory>
+#include <optional>
+#include <string>
+#include <utility>
+#include <functional>
+#include <mutex>
+#include <condition_variable>
+
+#include "core/logging/Logger.h"
+#include "concurrentqueue.h"
+#include "MinifiConcurrentQueue.h"
+
+namespace org::apache::nifi::minifi::utils {
+
+template<class ResourceType>
+class ResourceQueue : public std::enable_shared_from_this<ResourceQueue<ResourceType>> {

Review Comment:
   This class is complex enough to warrant a detailed description. I prefer reading 4 lines of English over 100 lines of advanced C++.



##########
extensions/http-curl/client/HTTPClient.cpp:
##########
@@ -378,10 +354,11 @@ int HTTPClient::onProgress(void *clientp, curl_off_t /*dltotal*/, curl_off_t dln
     return 0;
   }
   // did not transfer data
-  if (elapsed.count() > client.read_timeout_ms_.count()) {
+  if (elapsed > client.read_timeout_) {
     // timeout
-    client.logger_->log_error("HTTP operation has been idle for %dms, limit (%dms) reached, terminating connection\n",
-      static_cast<int>(elapsed.count()), static_cast<int>(client.read_timeout_ms_.count()));
+    client.logger_->log_error("HTTP operation has been idle for %" PRId64 " ms, limit (%" PRId64 "ms) reached, terminating connection\n",
+                              std::chrono::duration_cast<std::chrono::milliseconds>(elapsed).count(),
+                              client.read_timeout_.count());

Review Comment:
   Since chrono doesn't guarantee that the returned integer type will be int64_t, we have to convert it. Please add a braced initializer-style conversion, so that it fails to compile if this ever loses precision in the future. (In the suggestion, I also changed the indentation length according to my preference, but I'm not insisting on changing that.)
   ```suggestion
           int64_t{std::chrono::duration_cast<std::chrono::milliseconds>(elapsed).count()},
           int64_t{client.read_timeout_.count()});
   ```



##########
extensions/http-curl/client/HTTPClient.h:
##########
@@ -270,48 +249,33 @@ class HTTPClient : public BaseHTTPClient, public core::Connectable {
 
   void configure_secure_connection(CURL *http_session);
 
-  bool isSecure(const std::string &url);
+  std::chrono::milliseconds getAbsoluteTimeout() const { return 3*read_timeout_; }
 
-  HTTPReadCallback content_;
+  utils::HTTPReadCallback content_{std::numeric_limits<size_t>::max()};
 
   std::shared_ptr<minifi::controllers::SSLContextService> ssl_context_service_;
   std::string url_;
-  std::chrono::milliseconds connect_timeout_ms_{30000};
-  // read timeout.
-  std::chrono::milliseconds read_timeout_ms_{30000};
-  char *content_type_str_{nullptr};
-  std::string content_type_;
-  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{CURLE_OK};
-
-  CURL* http_session_{nullptr};
-  curl_mime* form_{nullptr};
-
   std::string method_;
 
-  std::chrono::milliseconds keep_alive_probe_{-1};
+  std::chrono::milliseconds connect_timeout_{std::chrono::seconds(30)};
+  std::chrono::milliseconds read_timeout_{std::chrono::seconds(30)};
 
-  std::chrono::milliseconds keep_alive_idle_{-1};
+  HTTPResponseData response_data_;
 
-  struct BasicAuthCredentials {
-    BasicAuthCredentials(std::string username, std::string password) : username(std::move(username)), password(std::move(password)) {}
+  CURLcode res_{CURLE_OK};
 
-    std::string username;
-    std::string password;
-  };
+  RequestHeaders request_headers_;
 
-  std::optional<BasicAuthCredentials> username_password_;
+  std::unique_ptr<CURL, decltype(&curl_easy_cleanup)> http_session_{nullptr, curl_easy_cleanup};
+  std::unique_ptr<curl_mime, decltype(&curl_mime_free)> form_{nullptr, curl_mime_free};

Review Comment:
   I generally prefer using types instead of function pointers for deleters. This allows three things:
   1. You can hide the name of the deleter function in the implementation file. We may not need to include curl in the header if it's not referenced elsewhere.
   2. You can use make_unique to initialize, since the Deleter type is default constructed instead of copied.
   3. The size of the unique_ptr can be reduced from two pointers (data and deleter function) to just one (data).



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

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

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


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

Posted by GitBox <gi...@apache.org>.
szaszm commented on PR #1383:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1383#issuecomment-1224655249

   pushed a rebase to main


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

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

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


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

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


##########
extensions/http-curl/processors/InvokeHTTP.h:
##########
@@ -127,52 +130,37 @@ class InvokeHTTP : public core::Processor {
 
   EXTENSIONAPI static constexpr const char* STATUS_CODE = "invokehttp.status.code";
   EXTENSIONAPI static constexpr const char* STATUS_MESSAGE = "invokehttp.status.message";
-  EXTENSIONAPI static constexpr const char* RESPONSE_BODY = "invokehttp.response.body";
   EXTENSIONAPI static constexpr const char* REQUEST_URL = "invokehttp.request.url";
   EXTENSIONAPI static constexpr const char* TRANSACTION_ID = "invokehttp.tx.id";
-  EXTENSIONAPI static constexpr const char* REMOTE_DN = "invokehttp.remote.dn";
-  EXTENSIONAPI static constexpr const char* EXCEPTION_CLASS = "invokehttp.java.exception.class";
-  EXTENSIONAPI static constexpr const char* EXCEPTION_MESSAGE = "invokehttp.java.exception.message";
 
-  void onTrigger(const std::shared_ptr<core::ProcessContext> &context, const std::shared_ptr<core::ProcessSession> &session) override;
+  void onTrigger(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSession>& session) override;
   void initialize() override;
-  void onSchedule(const std::shared_ptr<core::ProcessContext> &context, const std::shared_ptr<core::ProcessSessionFactory> &sessionFactory) override;
+  void onSchedule(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSessionFactory>& sessionFactory) override;
 
  private:
-  /**
-   * Routes the flowfile to the proper destination
-   * @param request request flow file record
-   * @param response response flow file record
-   * @param session process session
-   * @param context process context
-   * @param isSuccess success code or not
-   * @param statuscode http response code.
-   */
-  void route(const std::shared_ptr<core::FlowFile> &request, const std::shared_ptr<core::FlowFile> &response, const std::shared_ptr<core::ProcessSession> &session,
-             const std::shared_ptr<core::ProcessContext> &context, bool is_success, int64_t status_code);
-  bool shouldEmitFlowFile() const;
+  void route(const std::shared_ptr<core::FlowFile>& request, const std::shared_ptr<core::FlowFile>& response, const std::shared_ptr<core::ProcessSession>& session,
+             const std::shared_ptr<core::ProcessContext>& context, bool is_success, int64_t status_code);
+  static bool shouldEmitFlowFile(minifi::extensions::curl::HTTPClient& client);
+  void onTriggerWithClient(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSession>& session, minifi::extensions::curl::HTTPClient& client);
   [[nodiscard]] bool appendHeaders(const core::FlowFile& flow_file, /*std::invocable<std::string, std::string>*/ auto append_header);
 
-  std::shared_ptr<minifi::controllers::SSLContextService> ssl_context_service_;
-  std::string method_;
-  std::string url_;
-  bool date_header_include_{true};
+
+  void setupMembersFromProperties(const core::ProcessContext& context);
+  std::unique_ptr<minifi::extensions::curl::HTTPClient> createHTTPClientFromPropertiesAndMembers(const core::ProcessContext& context) const;
+
   std::optional<utils::Regex> attributes_to_send_;
-  std::chrono::milliseconds connect_timeout_ms_{20000};
-  std::chrono::milliseconds read_timeout_ms_{20000};
-  // attribute in which response body will be added
-  std::string put_attribute_name_;
+
+  std::optional<std::string> put_response_body_in_attribute_;
   bool always_output_response_{false};
-  std::string content_type_;
   bool use_chunked_encoding_{false};
   bool penalize_no_retry_{false};
-  // disabling peer verification makes susceptible for MITM attacks
-  bool disable_peer_verification_{false};
-  utils::HTTPProxy proxy_;
-  bool follow_redirects_{true};
   bool send_body_{true};
+
   InvalidHTTPHeaderFieldHandlingOption invalid_http_header_field_handling_strategy_;
+
   std::shared_ptr<core::logging::Logger> logger_{core::logging::LoggerFactory<InvokeHTTP>::getLogger()};
+  gsl::not_null<std::shared_ptr<utils::ResourceQueue<extensions::curl::HTTPClient>>> client_queue_ = gsl::make_not_null(
+      utils::ResourceQueue<extensions::curl::HTTPClient>::create(getMaxConcurrentTasks(), logger_));

Review Comment:
   A dummy object with incorrect data is easier to misuse, than null. I agree.



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

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

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


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

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


##########
libminifi/test/unit/ResourceQueueTests.cpp:
##########
@@ -0,0 +1,90 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include <chrono>
+#include <set>
+
+#include "ResourceQueue.h"
+#include "../TestBase.h"
+#include "../Catch.h"
+#include "logging/LoggerConfiguration.h"
+
+using namespace std::literals::chrono_literals;
+
+namespace org::apache::nifi::minifi::utils::testing {
+
+TEST_CASE("maximum_number_of_creatable_resources", "[utils::ResourceQueue]") {
+  std::shared_ptr<core::logging::Logger> logger_{core::logging::LoggerFactory<ResourceQueue<int>>::getLogger()};
+  LogTestController::getInstance().setTrace<ResourceQueue<int>>();
+
+  std::set<int> resources_created;
+
+  auto worker = [&](int value, const std::shared_ptr<ResourceQueue<int>>& resource_queue) {
+    auto resource = resource_queue->getResource([value]{return std::make_unique<int>(value);});
+    std::this_thread::sleep_for(10ms);
+    resources_created.emplace(*resource);
+  };
+
+  SECTION("Maximum 2 resources") {
+    auto resource_queue = ResourceQueue<int>::create(2, logger_);
+    std::thread thread_one{[&] { worker(1, resource_queue); }};
+    std::thread thread_two{[&] { worker(2, resource_queue); }};
+    std::thread thread_three{[&] { worker(3, resource_queue); }};
+
+    thread_one.join();
+    thread_two.join();
+    thread_three.join();
+
+    CHECK(!resources_created.empty());
+    CHECK(resources_created.size() <= 2);
+  }
+
+
+  SECTION("No Maximum resources") {
+    auto resource_queue = ResourceQueue<int>::create(2, logger_);
+    std::thread thread_one{[&] { worker(1, resource_queue); }};
+    std::thread thread_two{[&] { worker(2, resource_queue); }};
+    std::thread thread_three{[&] { worker(3, resource_queue); }};
+
+    thread_one.join();
+    thread_two.join();
+    thread_three.join();
+
+    CHECK(!resources_created.empty());
+    CHECK(resources_created.size() <= 3);

Review Comment:
   I've added a log check that makes sure no waiting occurred. (this failed without the typo fix)
   https://github.com/apache/nifi-minifi-cpp/pull/1383/commits/3f4aeec40af42fb9df1b15fbf0c2298664e77675#diff-18dc99071b15b546122f8e94bbfd19397b0e4c5d1d9d1d385234b19c84881c76R69



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

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

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


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

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


##########
extensions/script/ExecuteScript.h:
##########
@@ -185,23 +124,11 @@ class ExecuteScript : public core::Processor {
 
   ScriptEngineFactory engine_factory_;
 #ifdef LUA_SUPPORT
-  std::unique_ptr<ScriptEngineQueue<lua::LuaScriptEngine>> script_engine_q_;
+  std::shared_ptr<utils::ResourceQueue<lua::LuaScriptEngine>> lua_script_engine_queue_;

Review Comment:
   Thanks, makes sense.



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

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

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


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

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


##########
libminifi/include/utils/BaseHTTPClient.h:
##########
@@ -51,38 +39,43 @@ struct HTTPProxy {
   int port = 0;
 };
 
-struct HTTPUploadCallback {

Review Comment:
   Would it may be possible to inherit from the ByteInputCallback instead of composition? It may shorten some [train wrecks](https://devcards.io/train-wreck) like `http_client_->getReadCallback()->getPtr()->getSize()`. Same for HTTPReadCallback



##########
extensions/http-curl/tests/unit/ConnectionCountingServer.h:
##########
@@ -0,0 +1,149 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#pragma once
+
+#include <array>
+#include <vector>
+#include <string>
+#include <set>
+#include "CivetServer.h"
+
+namespace org::apache::nifi::minifi::extensions::curl::testing {
+
+namespace details {
+
+class NumberedMethodResponder : public CivetHandler {
+ public:
+  explicit NumberedMethodResponder(std::set<utils::SmallString<36>>& connections) : connections_(connections) {}
+
+  bool handleGet(CivetServer*, struct mg_connection* conn) override {
+    sendNumberedMessage("GET", conn);
+    return true;
+  }
+
+  bool handlePost(CivetServer*, struct mg_connection* conn) override {
+    sendNumberedMessage("POST", conn);
+    return true;
+  }
+
+  bool handlePut(CivetServer*, struct mg_connection* conn) override {
+    sendNumberedMessage("PUT", conn);
+    return true;
+  }
+
+  bool handleHead(CivetServer*, struct mg_connection* conn) override {
+    sendNumberedMessage("HEAD", conn);
+    return true;
+  }
+
+ private:
+  void sendNumberedMessage(std::string body, struct mg_connection* conn) {
+    saveConnectionId(conn);
+    body.append(std::to_string(response_id_));
+    mg_printf(conn, "HTTP/1.1 200 OK\r\n");
+    mg_printf(conn, "Content-length: %lu\r\n", body.length());
+    mg_printf(conn, "Response-number: %" PRIu64 "\r\n", response_id_);
+    mg_printf(conn, "\r\n");
+    mg_printf(conn, body.data(), body.length());
+    ++response_id_;
+  }
+
+  void saveConnectionId(struct mg_connection* conn) {
+    auto user_connection_data = reinterpret_cast<utils::SmallString<36>*>(mg_get_user_connection_data(conn));
+    connections_.emplace(*user_connection_data);
+  }
+
+  uint64_t response_id_ = 0;
+  std::set<utils::SmallString<36>>& connections_;
+};
+
+class ReverseBodyPostHandler : public CivetHandler {
+ public:
+  explicit ReverseBodyPostHandler(std::set<utils::SmallString<36>>& connections) : connections_(connections) {}
+
+  bool handlePost(CivetServer* /*server*/, struct mg_connection* conn) override {
+    saveConnectionId(conn);
+    std::vector<char> request_body;
+    request_body.reserve(2048);
+    size_t read_size = mg_read(conn, request_body.data(), 2048);
+    std::string response_body{request_body.begin(), request_body.begin() + read_size};
+    std::reverse(std::begin(response_body), std::end(response_body));
+    mg_printf(conn, "HTTP/1.1 200 OK\r\n");
+    mg_printf(conn, "Content-length: %zu\r\n", read_size);
+    mg_printf(conn, "\r\n");
+    mg_printf(conn, response_body.data(), read_size);
+
+    return true;
+  }
+
+ private:
+  void saveConnectionId(struct mg_connection* conn) {
+    auto user_connection_data = reinterpret_cast<utils::SmallString<36>*>(mg_get_user_connection_data(conn));
+    connections_.emplace(*user_connection_data);
+  }
+
+  std::set<utils::SmallString<36>>& connections_;
+};
+
+struct AddIdToUserConnectionData : public CivetCallbacks {
+  AddIdToUserConnectionData() {
+    init_connection = [](const struct mg_connection*, void** user_connection_data) -> int {
+      utils::SmallString<36>* id = new utils::SmallString<36>(utils::IdGenerator::getIdGenerator()->generate().to_string());
+      *user_connection_data = reinterpret_cast<void*>(id);
+      return 0;
+    };
+
+    connection_close = [](const struct mg_connection* conn) -> void {
+      auto user_connection_data = reinterpret_cast<utils::SmallString<36>*>(mg_get_user_connection_data(conn));
+      delete user_connection_data;
+    };

Review Comment:
   These lambdas seem to be unused, am I missing something here?



##########
extensions/http-curl/client/HTTPStream.cpp:
##########
@@ -26,55 +26,56 @@
 #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->getPtr()->close();
+  if (auto upload_callback = http_client_->getUploadCallback())
+    upload_callback->getPtr()->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<utils::HTTPUploadCallback>(new HttpStreamingCallback());
+      callback->pos = 0;
+      http_client_->setUploadCallback(std::move(callback));
       http_client_future_ = std::async(std::launch::async, submit_client, http_client_);
       started_ = true;
     }
   }
-  http_callback_.process(value, size);
+  auto http_callback = dynamic_cast<HttpStreamingCallback*>(gsl::as_nullable(http_client_->getUploadCallback()->getPtr()));
+  if (http_callback)

Review Comment:
   These could be merged



##########
libminifi/include/utils/BaseHTTPClient.h:
##########
@@ -51,38 +39,43 @@ 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;
+class HTTPUploadCallback {
+ public:
+  explicit HTTPUploadCallback(ByteInputCallback* byte_input_callback) : ptr(std::move(byte_input_callback)) {}
 
   size_t getPos() {
     std::lock_guard<std::mutex> lock(mutex);
     return pos;

Review Comment:
   Could the mutex be removed and `atomic<size_t> pos;` used instead? Same for HTTPReadCallback



##########
extensions/http-curl/client/HTTPStream.cpp:
##########
@@ -26,55 +26,56 @@
 #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->getPtr()->close();
+  if (auto upload_callback = http_client_->getUploadCallback())
+    upload_callback->getPtr()->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<utils::HTTPUploadCallback>(new HttpStreamingCallback());

Review Comment:
   This could be simplified to `auto callback = std::make_unique<utils::HttpStreamingCallback>();`



##########
docker/test/integration/MiNiFi_integration_test_driver.py:
##########
@@ -132,7 +132,9 @@ def generate_input_port_for_remote_process_group(remote_process_group, name):
         input_port_node.set_uuid(uuid.uuid3(remote_process_group.get_uuid(), "input_port"))
         return input_port_node
 
-    def add_test_data(self, path, test_data, file_name=str(uuid.uuid4())):
+    def add_test_data(self, path, test_data, file_name=None):
+        if file_name is None:
+            file_name = str(uuid.uuid4())

Review Comment:
   What's the difference with this change?



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

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

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


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

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


##########
docker/test/integration/MiNiFi_integration_test_driver.py:
##########
@@ -132,7 +132,9 @@ def generate_input_port_for_remote_process_group(remote_process_group, name):
         input_port_node.set_uuid(uuid.uuid3(remote_process_group.get_uuid(), "input_port"))
         return input_port_node
 
-    def add_test_data(self, path, test_data, file_name=str(uuid.uuid4())):
+    def add_test_data(self, path, test_data, file_name=None):
+        if file_name is None:
+            file_name = str(uuid.uuid4())

Review Comment:
   It turns out in Python the default argument expression isn't evaluated when you call the function, it's evaluated when you create the function. (i.e. once)
   Before this change the random file_name was the same every-time you called the function



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

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

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


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

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


##########
libminifi/include/utils/BaseHTTPClient.h:
##########
@@ -51,38 +39,30 @@ 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;
+class HTTPUploadCallback : public ByteInputCallback {
+ public:
+  template<typename... Args>
+  explicit HTTPUploadCallback(Args&& ... args) : ByteInputCallback(std::forward<Args>(args)...) {}
 
   size_t getPos() {
-    std::lock_guard<std::mutex> lock(mutex);
     return pos;
   }

Review Comment:
   makes sense, removed them in https://github.com/apache/nifi-minifi-cpp/pull/1383/commits/b80c1c4c6adee5dba02fe5bd355f8732769edf99



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

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

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


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

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


##########
extensions/http-curl/processors/InvokeHTTP.cpp:
##########
@@ -83,165 +84,188 @@ const core::Property InvokeHTTP::ProxyUsername(
     core::PropertyBuilder::createProperty("invokehttp-proxy-username", "Proxy Username")->withDescription("Username to set when authenticating against proxy")->isRequired(false)->build());
 const core::Property InvokeHTTP::ProxyPassword(
     core::PropertyBuilder::createProperty("invokehttp-proxy-password", "Proxy Password")->withDescription("Password to set when authenticating against proxy")->isRequired(false)->build());
-const core::Property InvokeHTTP::ContentType("Content-type", "The Content-Type to specify for when content is being transmitted through a PUT, "
-                                       "POST or PATCH. In the case of an empty value after evaluating an expression language expression, "
-                                       "Content-Type defaults to",
-                                       "application/octet-stream");
+const core::Property InvokeHTTP::ContentType("Content-type",
+    "The Content-Type to specify for when content is being transmitted through a PUT, "
+    "POST or PATCH. In the case of an empty value after evaluating an expression language expression, "
+    "Content-Type defaults to",
+    "application/octet-stream");
 const core::Property InvokeHTTP::SendBody(
     core::PropertyBuilder::createProperty("send-message-body", "Send Body")
-      ->withDescription("DEPRECATED. Only kept for backwards compatibility, no functionality is included.")
-      ->withDefaultValue<bool>(true)
-      ->build());
+        ->withDescription("DEPRECATED. Only kept for backwards compatibility, no functionality is included.")
+        ->withDefaultValue<bool>(true)
+        ->build());
 const core::Property InvokeHTTP::SendMessageBody(
     core::PropertyBuilder::createProperty("Send Message Body")
-      ->withDescription("If true, sends the HTTP message body on POST/PUT/PATCH requests (default). "
-                        "If false, suppresses the message body and content-type header for these requests.")
-      ->withDefaultValue<bool>(true)
-      ->build());
-const core::Property InvokeHTTP::UseChunkedEncoding("Use Chunked Encoding", "When POST'ing, PUT'ing or PATCH'ing content set this property to true in order to not pass the 'Content-length' header"
-                                              " and instead send 'Transfer-Encoding' with a value of 'chunked'. This will enable the data transfer mechanism which was introduced in HTTP 1.1 "
-                                              "to pass data of unknown lengths in chunks.",
-                                              "false");
-const core::Property InvokeHTTP::PropPutOutputAttributes("Put Response Body in Attribute", "If set, the response body received back will be put into an attribute of the original "
-                                                   "FlowFile instead of a separate FlowFile. The attribute key to put to is determined by evaluating value of this property. ",
-                                                   "");
-const core::Property InvokeHTTP::AlwaysOutputResponse("Always Output Response", "Will force a response FlowFile to be generated and routed to the 'Response' relationship "
-                                                "regardless of what the server status code received is ",
-                                                "false");
-const core::Property InvokeHTTP::PenalizeOnNoRetry("Penalize on \"No Retry\"", "Enabling this property will penalize FlowFiles that are routed to the \"No Retry\" relationship.", "false");
+        ->withDescription("If true, sends the HTTP message body on POST/PUT/PATCH requests (default). "
+                          "If false, suppresses the message body and content-type header for these requests.")
+        ->withDefaultValue<bool>(true)
+        ->build());
+const core::Property InvokeHTTP::UseChunkedEncoding("Use Chunked Encoding",
+    "When POST'ing, PUT'ing or PATCH'ing content set this property to true in order to not pass the 'Content-length' header"
+    " and instead send 'Transfer-Encoding' with a value of 'chunked'."
+    " This will enable the data transfer mechanism which was introduced in HTTP 1.1 to pass data of unknown lengths in chunks.",
+    "false");
+const core::Property InvokeHTTP::PutResponseBodyInAttribute("Put Response Body in Attribute",
+    "If set, the response body received back will be put into an attribute of the original "
+    "FlowFile instead of a separate FlowFile. "
+    "The attribute key to put to is determined by evaluating value of this property. ",
+    "");
+const core::Property InvokeHTTP::AlwaysOutputResponse("Always Output Response",
+    "Will force a response FlowFile to be generated and routed to the 'Response' relationship regardless of what the server status code received is ",
+    "false");
+const core::Property InvokeHTTP::PenalizeOnNoRetry("Penalize on \"No Retry\"",
+    "Enabling this property will penalize FlowFiles that are routed to the \"No Retry\" relationship.",
+    "false");
 
 const core::Property InvokeHTTP::DisablePeerVerification("Disable Peer Verification", "Disables peer verification for the SSL session", "false");
 
 const core::Property InvokeHTTP::InvalidHTTPHeaderFieldHandlingStrategy(
     core::PropertyBuilder::createProperty("Invalid HTTP Header Field Handling Strategy")
-      ->withDescription("Indicates what should happen when an attribute's name is not a valid HTTP header field name. "
-        "Options: transform - invalid characters are replaced, fail - flow file is transferred to failure, drop - drops invalid attributes from HTTP message")
-      ->isRequired(true)
-      ->withDefaultValue<std::string>(toString(InvalidHTTPHeaderFieldHandlingOption::TRANSFORM))
-      ->withAllowableValues<std::string>(InvalidHTTPHeaderFieldHandlingOption::values())
-      ->build());
+        ->withDescription("Indicates what should happen when an attribute's name is not a valid HTTP header field name. "
+                          "Options: transform - invalid characters are replaced, fail - flow file is transferred to failure, drop - drops invalid attributes from HTTP message")
+        ->isRequired(true)
+        ->withDefaultValue<std::string>(toString(InvalidHTTPHeaderFieldHandlingOption::TRANSFORM))
+        ->withAllowableValues<std::string>(InvalidHTTPHeaderFieldHandlingOption::values())
+        ->build());
 
 
-const core::Relationship InvokeHTTP::Success("success", "The original FlowFile will be routed upon success (2xx status codes). "
-                                       "It will have new attributes detailing the success of the request.");
+const core::Relationship InvokeHTTP::Success("success",
+    "The original FlowFile will be routed upon success (2xx status codes). It will have new attributes detailing the success of the request.");
 
-const core::Relationship InvokeHTTP::RelResponse("response", "A Response FlowFile will be routed upon success (2xx status codes). "
-                                           "If the 'Always Output Response' property is true then the response will be sent "
-                                           "to this relationship regardless of the status code received.");
+const core::Relationship InvokeHTTP::RelResponse("response",
+    "A Response FlowFile will be routed upon success (2xx status codes). "
+    "If the 'Always Output Response' property is true then the response will be sent "
+    "to this relationship regardless of the status code received.");
 
-const core::Relationship InvokeHTTP::RelRetry("retry", "The original FlowFile will be routed on any status code that can be retried "
-                                        "(5xx status codes). It will have new attributes detailing the request.");
+const core::Relationship InvokeHTTP::RelRetry("retry",
+    "The original FlowFile will be routed on any status code that can be retried "
+    "(5xx status codes). It will have new attributes detailing the request.");
 
-const core::Relationship InvokeHTTP::RelNoRetry("no retry", "The original FlowFile will be routed on any status code that should NOT "
-                                          "be retried (1xx, 3xx, 4xx status codes). It will have new attributes detailing the request.");
+const core::Relationship InvokeHTTP::RelNoRetry("no retry",
+    "The original FlowFile will be routed on any status code that should NOT "
+    "be retried (1xx, 3xx, 4xx status codes). It will have new attributes detailing the request.");
 
-const core::Relationship InvokeHTTP::RelFailure("failure", "The original FlowFile will be routed on any type of connection failure, "
-                                          "timeout or general exception. It will have new attributes detailing the request.");
+const core::Relationship InvokeHTTP::RelFailure("failure",
+    "The original FlowFile will be routed on any type of connection failure, "
+    "timeout or general exception. It will have new attributes detailing the request.");
 
 void InvokeHTTP::initialize() {
   logger_->log_trace("Initializing InvokeHTTP");
   setSupportedProperties(properties());
   setSupportedRelationships(relationships());
 }
 
-void InvokeHTTP::onSchedule(const std::shared_ptr<core::ProcessContext> &context, const std::shared_ptr<core::ProcessSessionFactory>& /*sessionFactory*/) {
-  if (!context->getProperty(Method.getName(), method_)) {
-    logger_->log_debug("%s attribute is missing, so default value of %s will be used", Method.getName(), Method.getValue());
-    return;
-  }
+namespace {
+void setupClientTimeouts(extensions::curl::HTTPClient& client, const core::ProcessContext& context) {
+  if (auto connection_timeout = context.getProperty<core::TimePeriodValue>(InvokeHTTP::ConnectTimeout))
+    client.setConnectionTimeout(connection_timeout->getMilliseconds());
 
-  if (!context->getProperty(URL.getName(), url_)) {
-    logger_->log_debug("%s attribute is missing, so default value of %s will be used", URL.getName(), URL.getValue());
-    return;
-  }
+  if (auto read_timeout = context.getProperty<core::TimePeriodValue>(InvokeHTTP::ReadTimeout))
+    client.setReadTimeout(read_timeout->getMilliseconds());
+}
 
-  if (auto connect_timeout = context->getProperty<core::TimePeriodValue>(ConnectTimeout)) {
-    connect_timeout_ms_ =  connect_timeout->getMilliseconds();
-  } else {
-    logger_->log_debug("%s attribute is missing, so default value of %s will be used", ConnectTimeout.getName(), ConnectTimeout.getValue());
-    return;
+void setupClientProxy(extensions::curl::HTTPClient& client, const core::ProcessContext& context) {
+  utils::HTTPProxy proxy = {};
+  context.getProperty(InvokeHTTP::ProxyHost.getName(), proxy.host);
+  std::string port_str;
+  if (context.getProperty(InvokeHTTP::ProxyPort.getName(), port_str) && !port_str.empty()) {
+    core::Property::StringToInt(port_str, proxy.port);
   }
+  context.getProperty(InvokeHTTP::ProxyUsername.getName(), proxy.username);
+  context.getProperty(InvokeHTTP::ProxyPassword.getName(), proxy.password);
 
-  std::string content_type_str;
-  if (context->getProperty(ContentType.getName(), content_type_str)) {
-    content_type_ = content_type_str;
-  }
+  client.setHTTPProxy(proxy);
+}
 
-  if (auto read_timeout = context->getProperty<core::TimePeriodValue>(ReadTimeout)) {
-    read_timeout_ms_ =  read_timeout->getMilliseconds();
-  } else {
-    logger_->log_debug("%s attribute is missing, so default value of %s will be used", ReadTimeout.getName(), ReadTimeout.getValue());
-  }
+void setupClientPeerVerification(extensions::curl::HTTPClient& client, const core::ProcessContext& context) {
+  if (auto disable_peer_verification = context.getProperty<bool>(InvokeHTTP::DisablePeerVerification))
+    client.setPeerVerification(*disable_peer_verification);
+}
+
+void setupClientFollowRedirects(extensions::curl::HTTPClient& client, const core::ProcessContext& context) {
+  if (auto follow_redirects = context.getProperty<bool>(InvokeHTTP::FollowRedirects))
+    client.setFollowRedirects(*follow_redirects);
+}
 
-  std::string date_header_str;
-  if (!context->getProperty(DateHeader.getName(), date_header_str)) {
-    logger_->log_debug("%s attribute is missing, so default value of %s will be used", DateHeader.getName(), DateHeader.getValue());
+void setupClientContentType(extensions::curl::HTTPClient& client, const core::ProcessContext& context, bool send_body) {
+  if (auto content_type = context.getProperty(InvokeHTTP::ContentType)) {
+    if (send_body)
+      client.setContentType(*content_type);
   }
+}
 
-  date_header_include_ = utils::StringUtils::toBool(date_header_str).value_or(DateHeader.getValue());
+void setupClientTransferEncoding(extensions::curl::HTTPClient& client, bool use_chunked_encoding) {
+  if (use_chunked_encoding)
+    client.setRequestHeader("Transfer-Encoding", "chunked");
+  else
+    client.setRequestHeader("Transfer-Encoding", std::nullopt);
+}
+}  // namespace
 
-  if (!context->getProperty(PropPutOutputAttributes.getName(), put_attribute_name_)) {
-    logger_->log_debug("%s attribute is missing, so default value of %s will be used", PropPutOutputAttributes.getName(), PropPutOutputAttributes.getValue());
-  }
+void InvokeHTTP::setupMembersFromProperties(const core::ProcessContext& context) {
+  context.getProperty(SendMessageBody.getName(), send_body_);

Review Comment:
   makes sense, I've renamed it in https://github.com/apache/nifi-minifi-cpp/pull/1383/commits/044ce269808a28e5a6eaeca34fb30747426a6ab7#diff-5880044259311bdd9e3a284e900759d3e34098641e21c89f78bdace17f4e63c7L157



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

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

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


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

Posted by GitBox <gi...@apache.org>.
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


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

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


##########
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:
   good idea, I've made the logging more informative in https://github.com/apache/nifi-minifi-cpp/pull/1383/commits/044ce269808a28e5a6eaeca34fb30747426a6ab7#diff-e83ea756e38995b3194ecc99d0a64fc19c8ef992fb7a575a34def27a67f7819aR192-R200



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

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

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


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

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


##########
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:
   absolutely not, removed the virtual keyword in https://github.com/apache/nifi-minifi-cpp/pull/1383/commits/044ce269808a28e5a6eaeca34fb30747426a6ab7#diff-475912c726caf957fa46ce1b55d3e845e89aa19b4c7caf2cd4a0ae1f860cbdd5L100



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

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

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


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

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


##########
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 removed it in https://github.com/apache/nifi-minifi-cpp/pull/1383/commits/044ce269808a28e5a6eaeca34fb30747426a6ab7#diff-4e08bc45485eac46e59652e031a79a51dca96dc3a08fa5bf6ee8d2b5603bda6eL32-L37



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

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

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


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

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


##########
extensions/http-curl/tests/unit/ConnectionCountingServer.h:
##########
@@ -0,0 +1,149 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#pragma once
+
+#include <array>
+#include <vector>
+#include <string>
+#include <set>
+#include "CivetServer.h"
+
+namespace org::apache::nifi::minifi::extensions::curl::testing {
+
+namespace details {
+
+class NumberedMethodResponder : public CivetHandler {
+ public:
+  explicit NumberedMethodResponder(std::set<utils::SmallString<36>>& connections) : connections_(connections) {}
+
+  bool handleGet(CivetServer*, struct mg_connection* conn) override {
+    sendNumberedMessage("GET", conn);
+    return true;
+  }
+
+  bool handlePost(CivetServer*, struct mg_connection* conn) override {
+    sendNumberedMessage("POST", conn);
+    return true;
+  }
+
+  bool handlePut(CivetServer*, struct mg_connection* conn) override {
+    sendNumberedMessage("PUT", conn);
+    return true;
+  }
+
+  bool handleHead(CivetServer*, struct mg_connection* conn) override {
+    sendNumberedMessage("HEAD", conn);
+    return true;
+  }
+
+ private:
+  void sendNumberedMessage(std::string body, struct mg_connection* conn) {
+    saveConnectionId(conn);
+    body.append(std::to_string(response_id_));
+    mg_printf(conn, "HTTP/1.1 200 OK\r\n");
+    mg_printf(conn, "Content-length: %lu\r\n", body.length());
+    mg_printf(conn, "Response-number: %" PRIu64 "\r\n", response_id_);
+    mg_printf(conn, "\r\n");
+    mg_printf(conn, body.data(), body.length());
+    ++response_id_;
+  }
+
+  void saveConnectionId(struct mg_connection* conn) {
+    auto user_connection_data = reinterpret_cast<utils::SmallString<36>*>(mg_get_user_connection_data(conn));
+    connections_.emplace(*user_connection_data);
+  }
+
+  uint64_t response_id_ = 0;
+  std::set<utils::SmallString<36>>& connections_;
+};
+
+class ReverseBodyPostHandler : public CivetHandler {
+ public:
+  explicit ReverseBodyPostHandler(std::set<utils::SmallString<36>>& connections) : connections_(connections) {}
+
+  bool handlePost(CivetServer* /*server*/, struct mg_connection* conn) override {
+    saveConnectionId(conn);
+    std::vector<char> request_body;
+    request_body.reserve(2048);
+    size_t read_size = mg_read(conn, request_body.data(), 2048);

Review Comment:
   I really meant `< 2048`, as `mg_read()` will not read more than its `len` parameter, so `<= 2048` is always true.  But if we get a full buffer of 2048 characters, that indicates that there may be more left to read, which would be a sign of something bad happening.



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

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

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


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

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


##########
docker/test/integration/steps/steps.py:
##########
@@ -271,6 +271,12 @@ def step_impl(context, content, path):
     context.test.add_test_data(path, content)
 
 
+@given("{number_of_files:d} files with the content \"{content}\" are present in \"{path}\"")
+def step_impl(context, number_of_files, content, path):

Review Comment:
   makes sense, fixed it in https://github.com/apache/nifi-minifi-cpp/pull/1383/commits/3edf5fcfd603463b67bd6c072fa0128bbb861787#diff-dce84c359f2f7128e501f5a322d8f6ac3325c2b471844f2c42cbef9abca58185R727



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

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

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


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

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


##########
extensions/http-curl/client/RequestHeaders.h:
##########
@@ -0,0 +1,58 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#pragma once
+
+#include <unordered_map>
+#include <memory>
+#include <optional>
+#include <string>
+#include <string_view>
+
+#ifdef WIN32
+#pragma comment(lib, "wldap32.lib" )
+#pragma comment(lib, "crypt32.lib" )
+#pragma comment(lib, "Ws2_32.lib")
+
+#define CURL_STATICLIB
+#include <curl/curl.h>
+#else
+#include <curl/curl.h>
+#endif
+
+namespace org::apache::nifi::minifi::extensions::curl {
+class RequestHeaders {
+ public:
+  RequestHeaders() = default;
+
+  void appendHeader(std::string key, std::string value);
+
+  void disableExpectHeader();
+
+  [[nodiscard]] std::unique_ptr<struct curl_slist, decltype(&curl_slist_free_all)> get() const;
+  [[nodiscard]] auto size() const { return headers_.size(); }
+  [[nodiscard]] bool empty() const;
+
+  std::string& operator[](const std::string& key);
+  std::string& operator[](std::string&& key);
+
+  [[nodiscard]] bool contains(const std::string& key) const;
+  void erase(const std::string& key);
+
+ private:
+  std::unordered_map<std::string, std::string> headers_;
+};

Review Comment:
   Not really, I played around with the idea of maintaining a curl_slist member and manipulate that directly with a map-like interface but later abandoned that idea because it was too complicated, and changed the class to contain only a map.
   
   Should I remove this and add a simple map+function? 



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

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

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


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

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


##########
extensions/http-curl/client/HTTPClient.cpp:
##########
@@ -164,212 +161,191 @@ 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::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) {

Review Comment:
   same here: I fixed it in https://github.com/apache/nifi-minifi-cpp/pull/1383/commits/3edf5fcfd603463b67bd6c072fa0128bbb861787#diff-475912c726caf957fa46ce1b55d3e845e89aa19b4c7caf2cd4a0ae1f860cbdd5R100



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

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

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


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

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


##########
extensions/http-curl/client/HTTPCallback.h:
##########
@@ -156,11 +152,11 @@ class HttpStreamingCallback final : public ByteInputCallback {
       current_pos_ = current_buffer_start_;
       total_bytes_loaded_ += current_vec_.size();
       logger_->log_trace("loadNextBuffer() loaded new buffer, ptr_: %p, size: %zu, current_buffer_start_: %zu, current_pos_: %zu, total_bytes_loaded_: %zu",
-          ptr_,
-          current_vec_.size(),
-          current_buffer_start_,
-          current_pos_,
-          total_bytes_loaded_);
+                         ptr_,
+                         current_vec_.size(),
+                         current_buffer_start_,
+                         current_pos_,
+                         total_bytes_loaded_);

Review Comment:
   fixed the indentation issues in https://github.com/apache/nifi-minifi-cpp/pull/1383/commits/3edf5fcfd603463b67bd6c072fa0128bbb861787



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

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

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


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

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


##########
extensions/http-curl/tests/unit/ConnectionCountingServer.h:
##########
@@ -0,0 +1,149 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#pragma once
+
+#include <array>
+#include <vector>
+#include <string>
+#include <set>
+#include "CivetServer.h"
+
+namespace org::apache::nifi::minifi::extensions::curl::testing {
+
+namespace details {
+
+class NumberedMethodResponder : public CivetHandler {
+ public:
+  explicit NumberedMethodResponder(std::set<utils::SmallString<36>>& connections) : connections_(connections) {}
+
+  bool handleGet(CivetServer*, struct mg_connection* conn) override {
+    sendNumberedMessage("GET", conn);
+    return true;
+  }
+
+  bool handlePost(CivetServer*, struct mg_connection* conn) override {
+    sendNumberedMessage("POST", conn);
+    return true;
+  }
+
+  bool handlePut(CivetServer*, struct mg_connection* conn) override {
+    sendNumberedMessage("PUT", conn);
+    return true;
+  }
+
+  bool handleHead(CivetServer*, struct mg_connection* conn) override {
+    sendNumberedMessage("HEAD", conn);
+    return true;
+  }
+
+ private:
+  void sendNumberedMessage(std::string body, struct mg_connection* conn) {
+    saveConnectionId(conn);
+    body.append(std::to_string(response_id_));
+    mg_printf(conn, "HTTP/1.1 200 OK\r\n");
+    mg_printf(conn, "Content-length: %lu\r\n", body.length());
+    mg_printf(conn, "Response-number: %" PRIu64 "\r\n", response_id_);
+    mg_printf(conn, "\r\n");
+    mg_printf(conn, body.data(), body.length());
+    ++response_id_;
+  }
+
+  void saveConnectionId(struct mg_connection* conn) {
+    auto user_connection_data = reinterpret_cast<utils::SmallString<36>*>(mg_get_user_connection_data(conn));
+    connections_.emplace(*user_connection_data);
+  }
+
+  uint64_t response_id_ = 0;
+  std::set<utils::SmallString<36>>& connections_;
+};
+
+class ReverseBodyPostHandler : public CivetHandler {
+ public:
+  explicit ReverseBodyPostHandler(std::set<utils::SmallString<36>>& connections) : connections_(connections) {}
+
+  bool handlePost(CivetServer* /*server*/, struct mg_connection* conn) override {
+    saveConnectionId(conn);
+    std::vector<char> request_body;
+    request_body.reserve(2048);
+    size_t read_size = mg_read(conn, request_body.data(), 2048);
+    std::string response_body{request_body.begin(), request_body.begin() + read_size};
+    std::reverse(std::begin(response_body), std::end(response_body));
+    mg_printf(conn, "HTTP/1.1 200 OK\r\n");
+    mg_printf(conn, "Content-length: %zu\r\n", read_size);
+    mg_printf(conn, "\r\n");
+    mg_printf(conn, response_body.data(), read_size);
+
+    return true;
+  }
+
+ private:
+  void saveConnectionId(struct mg_connection* conn) {
+    auto user_connection_data = reinterpret_cast<utils::SmallString<36>*>(mg_get_user_connection_data(conn));
+    connections_.emplace(*user_connection_data);
+  }
+
+  std::set<utils::SmallString<36>>& connections_;
+};
+
+struct AddIdToUserConnectionData : public CivetCallbacks {
+  AddIdToUserConnectionData() {
+    init_connection = [](const struct mg_connection*, void** user_connection_data) -> int {
+      utils::SmallString<36>* id = new utils::SmallString<36>(utils::IdGenerator::getIdGenerator()->generate().to_string());
+      *user_connection_data = reinterpret_cast<void*>(id);
+      return 0;
+    };
+
+    connection_close = [](const struct mg_connection* conn) -> void {
+      auto user_connection_data = reinterpret_cast<utils::SmallString<36>*>(mg_get_user_connection_data(conn));
+      delete user_connection_data;
+    };

Review Comment:
   The init_connection and connection_close are members of the parent class CivetCallbacks they are used somewhere inside CivetServer (we pass an instance of this struct to CivetServer `server_{options, &add_id_to_user_connection_data_};` )



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

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

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


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

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


##########
libminifi/include/utils/BaseHTTPClient.h:
##########
@@ -51,38 +39,43 @@ 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;
+class HTTPUploadCallback {
+ public:
+  explicit HTTPUploadCallback(ByteInputCallback* byte_input_callback) : ptr(std::move(byte_input_callback)) {}
 
   size_t getPos() {
     std::lock_guard<std::mutex> lock(mutex);
     return pos;

Review Comment:
   good idea, reworked them in https://github.com/apache/nifi-minifi-cpp/pull/1383/commits/4263282eb39be5757f303a61bb0d5380be157d66



##########
libminifi/include/utils/BaseHTTPClient.h:
##########
@@ -51,38 +39,43 @@ struct HTTPProxy {
   int port = 0;
 };
 
-struct HTTPUploadCallback {

Review Comment:
   good idea, reworked them in https://github.com/apache/nifi-minifi-cpp/pull/1383/commits/4263282eb39be5757f303a61bb0d5380be157d66



##########
extensions/http-curl/client/HTTPStream.cpp:
##########
@@ -26,55 +26,56 @@
 #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->getPtr()->close();
+  if (auto upload_callback = http_client_->getUploadCallback())
+    upload_callback->getPtr()->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<utils::HTTPUploadCallback>(new HttpStreamingCallback());
+      callback->pos = 0;
+      http_client_->setUploadCallback(std::move(callback));
       http_client_future_ = std::async(std::launch::async, submit_client, http_client_);
       started_ = true;
     }
   }
-  http_callback_.process(value, size);
+  auto http_callback = dynamic_cast<HttpStreamingCallback*>(gsl::as_nullable(http_client_->getUploadCallback()->getPtr()));
+  if (http_callback)

Review Comment:
   good idea, merged them in https://github.com/apache/nifi-minifi-cpp/commit/4263282eb39be5757f303a61bb0d5380be157d66#diff-4e08bc45485eac46e59652e031a79a51dca96dc3a08fa5bf6ee8d2b5603bda6eR74



##########
extensions/http-curl/client/HTTPStream.cpp:
##########
@@ -26,55 +26,56 @@
 #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->getPtr()->close();
+  if (auto upload_callback = http_client_->getUploadCallback())
+    upload_callback->getPtr()->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<utils::HTTPUploadCallback>(new HttpStreamingCallback());

Review Comment:
   now that it is inheritance I could make it simpler https://github.com/apache/nifi-minifi-cpp/commit/4263282eb39be5757f303a61bb0d5380be157d66#diff-4e08bc45485eac46e59652e031a79a51dca96dc3a08fa5bf6ee8d2b5603bda6eR67



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

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

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


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

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


##########
libminifi/include/utils/ResourceQueue.h:
##########
@@ -0,0 +1,132 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#pragma once
+
+#include <cstdint>
+#include <memory>
+#include <optional>
+#include <string>
+#include <utility>
+#include <functional>
+#include <mutex>
+#include <condition_variable>
+
+#include "core/logging/Logger.h"
+#include "concurrentqueue.h"
+#include "MinifiConcurrentQueue.h"
+
+namespace org::apache::nifi::minifi::utils {
+
+/*
+ * utils::ResourceQueue a threadsafe resource pool that lends out existing resources or creates them if necessary.
+ * getResource will return an existing unused resource or use the create_resource function to create one.
+ * If the number of existing resources reached the maximum_number_of_creatable_resources_, the call will block until a resource is returned to the queue.
+ * The lent out resource is in a ResourceWrapper that returns the resource to the queue on destruction.
+ * */
+
+template<class ResourceType>
+class ResourceQueue : public std::enable_shared_from_this<ResourceQueue<ResourceType>> {
+ public:
+  class ResourceWrapper {
+   public:
+    ResourceWrapper(std::weak_ptr<ResourceQueue<ResourceType>> queue, std::unique_ptr<ResourceType>&& resource) : queue_(std::move(queue)), resource_(std::move(resource)) {}
+    ResourceWrapper(ResourceWrapper&& src) = default;
+    ResourceWrapper(const ResourceWrapper&) = delete;
+    ~ResourceWrapper() {
+      if (auto queue = queue_.lock())
+        queue->returnResource(std::move(resource_));
+    }
+
+    ResourceWrapper& operator=(ResourceWrapper&&) = default;
+    ResourceWrapper& operator=(const ResourceWrapper&) = delete;
+
+    ResourceType& operator*() const { return *resource_; }
+    ResourceType* operator->() const noexcept { return resource_.operator->(); }
+    ResourceType* get() const { return resource_.get(); }
+
+   private:
+    std::weak_ptr<ResourceQueue<ResourceType>> queue_;
+    std::unique_ptr<ResourceType> resource_;
+  };
+
+  static auto create(std::optional<size_t> maximum_number_of_creatable_resources, std::shared_ptr<core::logging::Logger> logger);
+
+  template<typename Fn>
+  [[nodiscard]] std::enable_if_t<std::is_invocable_v<std::unique_ptr<ResourceType>()>, ResourceWrapper> getResource(const Fn& create_resource) {
+    std::unique_ptr<ResourceType> resource;
+    // Use an existing resource, if one is available
+    if (internal_queue_.tryDequeue(resource)) {
+      logDebug("Using available [%p] resource instance", resource.get());
+      return ResourceWrapper(this->weak_from_this(), std::move(resource));
+    } else {
+      const std::lock_guard<std::mutex> lock(counter_mutex_);
+      if (!maximum_number_of_creatable_resources_ || resources_created_ < maximum_number_of_creatable_resources_) {
+        ++resources_created_;
+        resource = create_resource();
+        logDebug("Created new [%p] resource instance. Number of instances: %d%s.",
+                 resource.get(),
+                 resources_created_,
+                 maximum_number_of_creatable_resources_ ? " / " + std::to_string(*maximum_number_of_creatable_resources_) : "");
+        return ResourceWrapper(this->weak_from_this(), std::move(resource));
+      }
+    }
+    logDebug("Waiting for resource");
+    if (!internal_queue_.dequeueWait(resource)) {
+      throw std::runtime_error("No resource available");
+    }
+    return ResourceWrapper(this->weak_from_this(), std::move(resource));
+  }
+
+ protected:
+  ResourceQueue(std::optional<size_t> maximum_number_of_creatable_resources, std::shared_ptr<core::logging::Logger> logger)
+      : maximum_number_of_creatable_resources_(maximum_number_of_creatable_resources),
+        logger_(std::move(logger)) {
+  }
+
+ private:
+  void returnResource(std::unique_ptr<ResourceType>&& resource) {

Review Comment:
   is this `&&` necessary?



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

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

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


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

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


##########
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.
   
   Also, we have range-v3, so I would write
   ```c++
     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


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

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


##########
extensions/http-curl/processors/InvokeHTTP.cpp:
##########
@@ -83,165 +84,188 @@ const core::Property InvokeHTTP::ProxyUsername(
     core::PropertyBuilder::createProperty("invokehttp-proxy-username", "Proxy Username")->withDescription("Username to set when authenticating against proxy")->isRequired(false)->build());
 const core::Property InvokeHTTP::ProxyPassword(
     core::PropertyBuilder::createProperty("invokehttp-proxy-password", "Proxy Password")->withDescription("Password to set when authenticating against proxy")->isRequired(false)->build());
-const core::Property InvokeHTTP::ContentType("Content-type", "The Content-Type to specify for when content is being transmitted through a PUT, "
-                                       "POST or PATCH. In the case of an empty value after evaluating an expression language expression, "
-                                       "Content-Type defaults to",
-                                       "application/octet-stream");
+const core::Property InvokeHTTP::ContentType("Content-type",
+    "The Content-Type to specify for when content is being transmitted through a PUT, "
+    "POST or PATCH. In the case of an empty value after evaluating an expression language expression, "
+    "Content-Type defaults to",
+    "application/octet-stream");
 const core::Property InvokeHTTP::SendBody(
     core::PropertyBuilder::createProperty("send-message-body", "Send Body")
-      ->withDescription("DEPRECATED. Only kept for backwards compatibility, no functionality is included.")
-      ->withDefaultValue<bool>(true)
-      ->build());
+        ->withDescription("DEPRECATED. Only kept for backwards compatibility, no functionality is included.")
+        ->withDefaultValue<bool>(true)
+        ->build());
 const core::Property InvokeHTTP::SendMessageBody(
     core::PropertyBuilder::createProperty("Send Message Body")
-      ->withDescription("If true, sends the HTTP message body on POST/PUT/PATCH requests (default). "
-                        "If false, suppresses the message body and content-type header for these requests.")
-      ->withDefaultValue<bool>(true)
-      ->build());
-const core::Property InvokeHTTP::UseChunkedEncoding("Use Chunked Encoding", "When POST'ing, PUT'ing or PATCH'ing content set this property to true in order to not pass the 'Content-length' header"
-                                              " and instead send 'Transfer-Encoding' with a value of 'chunked'. This will enable the data transfer mechanism which was introduced in HTTP 1.1 "
-                                              "to pass data of unknown lengths in chunks.",
-                                              "false");
-const core::Property InvokeHTTP::PropPutOutputAttributes("Put Response Body in Attribute", "If set, the response body received back will be put into an attribute of the original "
-                                                   "FlowFile instead of a separate FlowFile. The attribute key to put to is determined by evaluating value of this property. ",
-                                                   "");
-const core::Property InvokeHTTP::AlwaysOutputResponse("Always Output Response", "Will force a response FlowFile to be generated and routed to the 'Response' relationship "
-                                                "regardless of what the server status code received is ",
-                                                "false");
-const core::Property InvokeHTTP::PenalizeOnNoRetry("Penalize on \"No Retry\"", "Enabling this property will penalize FlowFiles that are routed to the \"No Retry\" relationship.", "false");
+        ->withDescription("If true, sends the HTTP message body on POST/PUT/PATCH requests (default). "
+                          "If false, suppresses the message body and content-type header for these requests.")
+        ->withDefaultValue<bool>(true)
+        ->build());
+const core::Property InvokeHTTP::UseChunkedEncoding("Use Chunked Encoding",
+    "When POST'ing, PUT'ing or PATCH'ing content set this property to true in order to not pass the 'Content-length' header"
+    " and instead send 'Transfer-Encoding' with a value of 'chunked'."
+    " This will enable the data transfer mechanism which was introduced in HTTP 1.1 to pass data of unknown lengths in chunks.",
+    "false");
+const core::Property InvokeHTTP::PutResponseBodyInAttribute("Put Response Body in Attribute",
+    "If set, the response body received back will be put into an attribute of the original "
+    "FlowFile instead of a separate FlowFile. "
+    "The attribute key to put to is determined by evaluating value of this property. ",
+    "");
+const core::Property InvokeHTTP::AlwaysOutputResponse("Always Output Response",
+    "Will force a response FlowFile to be generated and routed to the 'Response' relationship regardless of what the server status code received is ",
+    "false");
+const core::Property InvokeHTTP::PenalizeOnNoRetry("Penalize on \"No Retry\"",
+    "Enabling this property will penalize FlowFiles that are routed to the \"No Retry\" relationship.",
+    "false");
 
 const core::Property InvokeHTTP::DisablePeerVerification("Disable Peer Verification", "Disables peer verification for the SSL session", "false");
 
 const core::Property InvokeHTTP::InvalidHTTPHeaderFieldHandlingStrategy(
     core::PropertyBuilder::createProperty("Invalid HTTP Header Field Handling Strategy")
-      ->withDescription("Indicates what should happen when an attribute's name is not a valid HTTP header field name. "
-        "Options: transform - invalid characters are replaced, fail - flow file is transferred to failure, drop - drops invalid attributes from HTTP message")
-      ->isRequired(true)
-      ->withDefaultValue<std::string>(toString(InvalidHTTPHeaderFieldHandlingOption::TRANSFORM))
-      ->withAllowableValues<std::string>(InvalidHTTPHeaderFieldHandlingOption::values())
-      ->build());
+        ->withDescription("Indicates what should happen when an attribute's name is not a valid HTTP header field name. "
+                          "Options: transform - invalid characters are replaced, fail - flow file is transferred to failure, drop - drops invalid attributes from HTTP message")
+        ->isRequired(true)
+        ->withDefaultValue<std::string>(toString(InvalidHTTPHeaderFieldHandlingOption::TRANSFORM))
+        ->withAllowableValues<std::string>(InvalidHTTPHeaderFieldHandlingOption::values())
+        ->build());
 
 
-const core::Relationship InvokeHTTP::Success("success", "The original FlowFile will be routed upon success (2xx status codes). "
-                                       "It will have new attributes detailing the success of the request.");
+const core::Relationship InvokeHTTP::Success("success",
+    "The original FlowFile will be routed upon success (2xx status codes). It will have new attributes detailing the success of the request.");
 
-const core::Relationship InvokeHTTP::RelResponse("response", "A Response FlowFile will be routed upon success (2xx status codes). "
-                                           "If the 'Always Output Response' property is true then the response will be sent "
-                                           "to this relationship regardless of the status code received.");
+const core::Relationship InvokeHTTP::RelResponse("response",
+    "A Response FlowFile will be routed upon success (2xx status codes). "
+    "If the 'Always Output Response' property is true then the response will be sent "
+    "to this relationship regardless of the status code received.");
 
-const core::Relationship InvokeHTTP::RelRetry("retry", "The original FlowFile will be routed on any status code that can be retried "
-                                        "(5xx status codes). It will have new attributes detailing the request.");
+const core::Relationship InvokeHTTP::RelRetry("retry",
+    "The original FlowFile will be routed on any status code that can be retried "
+    "(5xx status codes). It will have new attributes detailing the request.");
 
-const core::Relationship InvokeHTTP::RelNoRetry("no retry", "The original FlowFile will be routed on any status code that should NOT "
-                                          "be retried (1xx, 3xx, 4xx status codes). It will have new attributes detailing the request.");
+const core::Relationship InvokeHTTP::RelNoRetry("no retry",
+    "The original FlowFile will be routed on any status code that should NOT "
+    "be retried (1xx, 3xx, 4xx status codes). It will have new attributes detailing the request.");
 
-const core::Relationship InvokeHTTP::RelFailure("failure", "The original FlowFile will be routed on any type of connection failure, "
-                                          "timeout or general exception. It will have new attributes detailing the request.");
+const core::Relationship InvokeHTTP::RelFailure("failure",
+    "The original FlowFile will be routed on any type of connection failure, "
+    "timeout or general exception. It will have new attributes detailing the request.");
 
 void InvokeHTTP::initialize() {
   logger_->log_trace("Initializing InvokeHTTP");
   setSupportedProperties(properties());
   setSupportedRelationships(relationships());
 }
 
-void InvokeHTTP::onSchedule(const std::shared_ptr<core::ProcessContext> &context, const std::shared_ptr<core::ProcessSessionFactory>& /*sessionFactory*/) {
-  if (!context->getProperty(Method.getName(), method_)) {
-    logger_->log_debug("%s attribute is missing, so default value of %s will be used", Method.getName(), Method.getValue());
-    return;
-  }
+namespace {
+void setupClientTimeouts(extensions::curl::HTTPClient& client, const core::ProcessContext& context) {
+  if (auto connection_timeout = context.getProperty<core::TimePeriodValue>(InvokeHTTP::ConnectTimeout))
+    client.setConnectionTimeout(connection_timeout->getMilliseconds());
 
-  if (!context->getProperty(URL.getName(), url_)) {
-    logger_->log_debug("%s attribute is missing, so default value of %s will be used", URL.getName(), URL.getValue());
-    return;
-  }
+  if (auto read_timeout = context.getProperty<core::TimePeriodValue>(InvokeHTTP::ReadTimeout))
+    client.setReadTimeout(read_timeout->getMilliseconds());
+}
 
-  if (auto connect_timeout = context->getProperty<core::TimePeriodValue>(ConnectTimeout)) {
-    connect_timeout_ms_ =  connect_timeout->getMilliseconds();
-  } else {
-    logger_->log_debug("%s attribute is missing, so default value of %s will be used", ConnectTimeout.getName(), ConnectTimeout.getValue());
-    return;
+void setupClientProxy(extensions::curl::HTTPClient& client, const core::ProcessContext& context) {
+  utils::HTTPProxy proxy = {};
+  context.getProperty(InvokeHTTP::ProxyHost.getName(), proxy.host);
+  std::string port_str;
+  if (context.getProperty(InvokeHTTP::ProxyPort.getName(), port_str) && !port_str.empty()) {
+    core::Property::StringToInt(port_str, proxy.port);
   }
+  context.getProperty(InvokeHTTP::ProxyUsername.getName(), proxy.username);
+  context.getProperty(InvokeHTTP::ProxyPassword.getName(), proxy.password);
 
-  std::string content_type_str;
-  if (context->getProperty(ContentType.getName(), content_type_str)) {
-    content_type_ = content_type_str;
-  }
+  client.setHTTPProxy(proxy);
+}
 
-  if (auto read_timeout = context->getProperty<core::TimePeriodValue>(ReadTimeout)) {
-    read_timeout_ms_ =  read_timeout->getMilliseconds();
-  } else {
-    logger_->log_debug("%s attribute is missing, so default value of %s will be used", ReadTimeout.getName(), ReadTimeout.getValue());
-  }
+void setupClientPeerVerification(extensions::curl::HTTPClient& client, const core::ProcessContext& context) {
+  if (auto disable_peer_verification = context.getProperty<bool>(InvokeHTTP::DisablePeerVerification))
+    client.setPeerVerification(*disable_peer_verification);
+}
+
+void setupClientFollowRedirects(extensions::curl::HTTPClient& client, const core::ProcessContext& context) {
+  if (auto follow_redirects = context.getProperty<bool>(InvokeHTTP::FollowRedirects))
+    client.setFollowRedirects(*follow_redirects);
+}
 
-  std::string date_header_str;
-  if (!context->getProperty(DateHeader.getName(), date_header_str)) {
-    logger_->log_debug("%s attribute is missing, so default value of %s will be used", DateHeader.getName(), DateHeader.getValue());
+void setupClientContentType(extensions::curl::HTTPClient& client, const core::ProcessContext& context, bool send_body) {
+  if (auto content_type = context.getProperty(InvokeHTTP::ContentType)) {
+    if (send_body)
+      client.setContentType(*content_type);
   }
+}
 
-  date_header_include_ = utils::StringUtils::toBool(date_header_str).value_or(DateHeader.getValue());
+void setupClientTransferEncoding(extensions::curl::HTTPClient& client, bool use_chunked_encoding) {
+  if (use_chunked_encoding)
+    client.setRequestHeader("Transfer-Encoding", "chunked");
+  else
+    client.setRequestHeader("Transfer-Encoding", std::nullopt);
+}
+}  // namespace
 
-  if (!context->getProperty(PropPutOutputAttributes.getName(), put_attribute_name_)) {
-    logger_->log_debug("%s attribute is missing, so default value of %s will be used", PropPutOutputAttributes.getName(), PropPutOutputAttributes.getValue());
-  }
+void InvokeHTTP::setupMembersFromProperties(const core::ProcessContext& context) {
+  context.getProperty(SendMessageBody.getName(), send_body_);
 
-  attributes_to_send_ = context->getProperty(AttributesToSend)
-      | utils::filter([](const std::string& s) { return !s.empty(); })  // avoid compiling an empty string to regex
-      | utils::map([](const std::string& regex_str) { return utils::Regex{regex_str}; })
-      | utils::orElse([this] { logger_->log_debug("%s is missing, so the default value will be used", AttributesToSend.getName()); });
+  attributes_to_send_ = context.getProperty(AttributesToSend)
+                        | utils::filter([](const std::string& s) { return !s.empty(); })  // avoid compiling an empty string to regex
+                        | utils::map([](const std::string& regex_str) { return utils::Regex{regex_str}; })
+                        | utils::orElse([this] { logger_->log_debug("%s is missing, so the default value will be used", AttributesToSend.getName()); });
 
-  std::string always_output_response;
-  if (!context->getProperty(AlwaysOutputResponse.getName(), always_output_response)) {
-    logger_->log_debug("%s attribute is missing, so default value of %s will be used", AlwaysOutputResponse.getName(), AlwaysOutputResponse.getValue());
-  }
 
-  always_output_response_ = utils::StringUtils::toBool(always_output_response).value_or(false);
+  always_output_response_ = context.getProperty<bool>(AlwaysOutputResponse).value_or(false);
+  penalize_no_retry_ = context.getProperty<bool>(PenalizeOnNoRetry).value_or(false);
 
-  std::string penalize_no_retry = "false";
-  if (!context->getProperty(PenalizeOnNoRetry.getName(), penalize_no_retry)) {
-    logger_->log_debug("%s attribute is missing, so default value of %s will be used", PenalizeOnNoRetry.getName(), PenalizeOnNoRetry.getValue());
-  }
+  invalid_http_header_field_handling_strategy_ = utils::parseEnumProperty<InvalidHTTPHeaderFieldHandlingOption>(context, InvalidHTTPHeaderFieldHandlingStrategy);
 
-  penalize_no_retry_ = utils::StringUtils::toBool(penalize_no_retry).value_or(false);
+  put_response_body_in_attribute_ = context.getProperty(PutResponseBodyInAttribute);
+  if (put_response_body_in_attribute_ && put_response_body_in_attribute_->empty()) {
+    logger_->log_warn("%s is set to an empty string", PutResponseBodyInAttribute.getName());
+    put_response_body_in_attribute_.reset();
+  }
+}
 
-  std::string context_name;
-  if (context->getProperty(SSLContext.getName(), context_name) && !IsNullOrEmpty(context_name)) {
-    std::shared_ptr<core::controller::ControllerService> service = context->getControllerService(context_name);
-    if (!service) {
-      logger_->log_error("Couldn't find controller service with name '%s'", context_name);
+std::unique_ptr<minifi::extensions::curl::HTTPClient> InvokeHTTP::createHTTPClientFromPropertiesAndMembers(const core::ProcessContext& context) const {
+  std::string method;
+  if (!context.getProperty(Method.getName(), method))
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Method property missing or invalid");
+
+  std::string url;
+  if (!context.getProperty(URL.getName(), url))
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "URL property missing or invalid");
+
+  std::shared_ptr<minifi::controllers::SSLContextService> ssl_context_service;
+  if (auto ssl_context_name = context.getProperty(SSLContext)) {
+    if (auto service = context.getControllerService(*ssl_context_name)) {
+      ssl_context_service = std::dynamic_pointer_cast<minifi::controllers::SSLContextService>(service);
+      if (!ssl_context_service)
+        logger_->log_error("Controller service '%s' is not an SSLContextService", *ssl_context_name);
     } else {
-      ssl_context_service_ = std::dynamic_pointer_cast<minifi::controllers::SSLContextService>(service);
-      if (!ssl_context_service_) {
-        logger_->log_error("Controller service '%s' is not an SSLContextService", context_name);
-      }
+      logger_->log_error("Couldn't find controller service with name '%s'", *ssl_context_name);
     }
   }
 
-  std::string use_chunked_encoding = "false";
-  if (!context->getProperty(UseChunkedEncoding.getName(), use_chunked_encoding)) {
-    logger_->log_debug("%s attribute is missing, so default value of %s will be used", UseChunkedEncoding.getName(), UseChunkedEncoding.getValue());
-  }
+  auto client = std::make_unique<minifi::extensions::curl::HTTPClient>();
+  client->initialize(std::move(method), std::move(url), std::move(ssl_context_service));
+  setupClientTimeouts(*client, context);
+  setupClientProxy(*client, context);
+  setupClientFollowRedirects(*client, context);
+  setupClientPeerVerification(*client, context);
+  setupClientContentType(*client, context, send_body_);
+  setupClientTransferEncoding(*client, use_chunked_encoding_);
 
-  use_chunked_encoding_ = utils::StringUtils::toBool(use_chunked_encoding).value_or(false);
+  return client;
+}
 
-  std::string disable_peer_verification;
-  disable_peer_verification_ = (context->getProperty(DisablePeerVerification.getName(), disable_peer_verification) && utils::StringUtils::toBool(disable_peer_verification).value_or(false));
 
-  proxy_ = {};
-  context->getProperty(ProxyHost.getName(), proxy_.host);
-  std::string port_str;
-  if (context->getProperty(ProxyPort.getName(), port_str) && !port_str.empty()) {
-    core::Property::StringToInt(port_str, proxy_.port);
-  }
-  context->getProperty(ProxyUsername.getName(), proxy_.username);
-  context->getProperty(ProxyPassword.getName(), proxy_.password);
-  context->getProperty(FollowRedirects.getName(), follow_redirects_);
-  context->getProperty(SendMessageBody.getName(), send_body_);
+void InvokeHTTP::onSchedule(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSessionFactory>& /*sessionFactory*/) {
+  gsl_Expects(context);
 
-  invalid_http_header_field_handling_strategy_ = utils::parseEnumProperty<InvalidHTTPHeaderFieldHandlingOption>(*context, InvalidHTTPHeaderFieldHandlingStrategy);
+  setupMembersFromProperties(*context);
+  client_queue_ = gsl::make_not_null(utils::ResourceQueue<extensions::curl::HTTPClient>::create(getMaxConcurrentTasks(), logger_));
 }
 
-bool InvokeHTTP::shouldEmitFlowFile() const {
-  return ("POST" == method_ || "PUT" == method_ || "PATCH" == method_);
+bool InvokeHTTP::shouldEmitFlowFile(minifi::extensions::curl::HTTPClient& client) {

Review Comment:
   its static now



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

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

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


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

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


##########
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:
   good idea, I've removed these in https://github.com/apache/nifi-minifi-cpp/pull/1383/commits/044ce269808a28e5a6eaeca34fb30747426a6ab7#diff-ba0ecec678efd02fccd01cdc5d4b580836c1d32d2608d277755742a404455b2dL55-L57



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

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

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


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

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


##########
extensions/script/ExecuteScript.h:
##########
@@ -185,23 +124,11 @@ class ExecuteScript : public core::Processor {
 
   ScriptEngineFactory engine_factory_;
 #ifdef LUA_SUPPORT
-  std::unique_ptr<ScriptEngineQueue<lua::LuaScriptEngine>> script_engine_q_;
+  std::shared_ptr<utils::ResourceQueue<lua::LuaScriptEngine>> lua_script_engine_queue_;

Review Comment:
   The utils::ResourceQueue must be a shared_ptr because the ResourceWrappers have weak_ptrs in them so they can return the resource (if the queue is still available).



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

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

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


[GitHub] [nifi-minifi-cpp] szaszm closed pull request #1383: MINIFICPP-1875 HTTPClient should be reusable

Posted by GitBox <gi...@apache.org>.
szaszm closed pull request #1383: MINIFICPP-1875 HTTPClient should be reusable
URL: https://github.com/apache/nifi-minifi-cpp/pull/1383


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

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

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


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

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


##########
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:
   not nitpicking at all, thanks I've included this in https://github.com/apache/nifi-minifi-cpp/pull/1383/commits/044ce269808a28e5a6eaeca34fb30747426a6ab7#diff-e83ea756e38995b3194ecc99d0a64fc19c8ef992fb7a575a34def27a67f7819aR362



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

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

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


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

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


##########
extensions/http-curl/processors/InvokeHTTP.h:
##########
@@ -127,52 +130,37 @@ class InvokeHTTP : public core::Processor {
 
   EXTENSIONAPI static constexpr const char* STATUS_CODE = "invokehttp.status.code";
   EXTENSIONAPI static constexpr const char* STATUS_MESSAGE = "invokehttp.status.message";
-  EXTENSIONAPI static constexpr const char* RESPONSE_BODY = "invokehttp.response.body";
   EXTENSIONAPI static constexpr const char* REQUEST_URL = "invokehttp.request.url";
   EXTENSIONAPI static constexpr const char* TRANSACTION_ID = "invokehttp.tx.id";
-  EXTENSIONAPI static constexpr const char* REMOTE_DN = "invokehttp.remote.dn";
-  EXTENSIONAPI static constexpr const char* EXCEPTION_CLASS = "invokehttp.java.exception.class";
-  EXTENSIONAPI static constexpr const char* EXCEPTION_MESSAGE = "invokehttp.java.exception.message";
 
-  void onTrigger(const std::shared_ptr<core::ProcessContext> &context, const std::shared_ptr<core::ProcessSession> &session) override;
+  void onTrigger(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSession>& session) override;
   void initialize() override;
-  void onSchedule(const std::shared_ptr<core::ProcessContext> &context, const std::shared_ptr<core::ProcessSessionFactory> &sessionFactory) override;
+  void onSchedule(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSessionFactory>& sessionFactory) override;
 
  private:
-  /**
-   * Routes the flowfile to the proper destination
-   * @param request request flow file record
-   * @param response response flow file record
-   * @param session process session
-   * @param context process context
-   * @param isSuccess success code or not
-   * @param statuscode http response code.
-   */
-  void route(const std::shared_ptr<core::FlowFile> &request, const std::shared_ptr<core::FlowFile> &response, const std::shared_ptr<core::ProcessSession> &session,
-             const std::shared_ptr<core::ProcessContext> &context, bool is_success, int64_t status_code);
-  bool shouldEmitFlowFile() const;
+  void route(const std::shared_ptr<core::FlowFile>& request, const std::shared_ptr<core::FlowFile>& response, const std::shared_ptr<core::ProcessSession>& session,
+             const std::shared_ptr<core::ProcessContext>& context, bool is_success, int64_t status_code);
+  static bool shouldEmitFlowFile(minifi::extensions::curl::HTTPClient& client);
+  void onTriggerWithClient(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSession>& session, minifi::extensions::curl::HTTPClient& client);
   [[nodiscard]] bool appendHeaders(const core::FlowFile& flow_file, /*std::invocable<std::string, std::string>*/ auto append_header);
 
-  std::shared_ptr<minifi::controllers::SSLContextService> ssl_context_service_;
-  std::string method_;
-  std::string url_;
-  bool date_header_include_{true};
+
+  void setupMembersFromProperties(const core::ProcessContext& context);
+  std::unique_ptr<minifi::extensions::curl::HTTPClient> createHTTPClientFromPropertiesAndMembers(const core::ProcessContext& context) const;
+
   std::optional<utils::Regex> attributes_to_send_;
-  std::chrono::milliseconds connect_timeout_ms_{20000};
-  std::chrono::milliseconds read_timeout_ms_{20000};
-  // attribute in which response body will be added
-  std::string put_attribute_name_;
+
+  std::optional<std::string> put_response_body_in_attribute_;
   bool always_output_response_{false};
-  std::string content_type_;
   bool use_chunked_encoding_{false};
   bool penalize_no_retry_{false};
-  // disabling peer verification makes susceptible for MITM attacks
-  bool disable_peer_verification_{false};
-  utils::HTTPProxy proxy_;
-  bool follow_redirects_{true};
   bool send_body_{true};
+
   InvalidHTTPHeaderFieldHandlingOption invalid_http_header_field_handling_strategy_;
+
   std::shared_ptr<core::logging::Logger> logger_{core::logging::LoggerFactory<InvokeHTTP>::getLogger()};
+  gsl::not_null<std::shared_ptr<utils::ResourceQueue<extensions::curl::HTTPClient>>> client_queue_ = gsl::make_not_null(
+      utils::ResourceQueue<extensions::curl::HTTPClient>::create(getMaxConcurrentTasks(), logger_));

Review Comment:
   I see -- this is only here to make `client_queue_` `not_null`.  I would prefer to leave it null at construction, and add a `gsl_Expects(client_queue_)` at the point where we use it, as constructing a dummy object with probably incorrect contents only to throw it away later feels wrong.  @szaszm may disagree, so just my 2¢.



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

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

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


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

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


##########
extensions/http-curl/tests/unit/InvokeHTTPTests.cpp:
##########
@@ -433,4 +414,76 @@ TEST_CASE("InvokeHTTP Attributes to Send uses full string matching, not substrin
   REQUIRE(LogTestController::getInstance().contains("key:header value:value2"));
   REQUIRE_FALSE(LogTestController::getInstance().contains("key:invalid", 0s));
 }
+
+TEST_CASE("HTTPTestsResponseBodyinAttribute", "[InvokeHTTP]") {
+  using minifi::processors::InvokeHTTP;
+
+  auto invoke_http = std::make_shared<InvokeHTTP>("InvokeHTTP");
+  test::SingleProcessorTestController test_controller{invoke_http};
+
+  minifi::extensions::curl::testing::ConnectionCountingServer connection_counting_server;
+
+  invoke_http->setProperty(InvokeHTTP::Method, "POST");
+  invoke_http->setProperty(InvokeHTTP::URL, "http://localhost:" + connection_counting_server.getPort()  + "/reverse");
+  invoke_http->setProperty(InvokeHTTP::PutResponseBodyInAttribute, "http.body");
+  const auto result = test_controller.trigger("data", {{"header1", "value1"}, {"header", "value2"}});
+  auto success_flow_files = result.at(InvokeHTTP::Success);
+  CHECK(result.at(InvokeHTTP::RelFailure).empty());
+  CHECK(result.at(InvokeHTTP::RelResponse).empty());
+  CHECK(result.at(InvokeHTTP::RelNoRetry).empty());
+  CHECK(result.at(InvokeHTTP::RelRetry).empty());
+  REQUIRE(success_flow_files.size() == 1);
+  CHECK(test_controller.plan->getContent(success_flow_files[0]) == "data");
+
+  auto http_type_attribute = success_flow_files[0]->getAttribute("http.body");
+  REQUIRE(http_type_attribute);
+  CHECK(*http_type_attribute == "atad");
+}
+
+TEST_CASE("HTTPTestsResponseBody", "[InvokeHTTP]") {
+  using minifi::processors::InvokeHTTP;
+
+  auto invoke_http = std::make_shared<InvokeHTTP>("InvokeHTTP");
+  test::SingleProcessorTestController test_controller{invoke_http};
+
+  minifi::extensions::curl::testing::ConnectionCountingServer connection_counting_server;
+
+  invoke_http->setProperty(InvokeHTTP::Method, "POST");
+  invoke_http->setProperty(InvokeHTTP::URL, "http://localhost:" + connection_counting_server.getPort()  + "/reverse");
+  invoke_http->setProperty(InvokeHTTP::SendBody, "true");

Review Comment:
   `SendBody` is deprecated, and it is ignored by the processor.  Should this be `SendMessageBody`?



##########
extensions/http-curl/processors/InvokeHTTP.cpp:
##########
@@ -83,165 +84,188 @@ const core::Property InvokeHTTP::ProxyUsername(
     core::PropertyBuilder::createProperty("invokehttp-proxy-username", "Proxy Username")->withDescription("Username to set when authenticating against proxy")->isRequired(false)->build());
 const core::Property InvokeHTTP::ProxyPassword(
     core::PropertyBuilder::createProperty("invokehttp-proxy-password", "Proxy Password")->withDescription("Password to set when authenticating against proxy")->isRequired(false)->build());
-const core::Property InvokeHTTP::ContentType("Content-type", "The Content-Type to specify for when content is being transmitted through a PUT, "
-                                       "POST or PATCH. In the case of an empty value after evaluating an expression language expression, "
-                                       "Content-Type defaults to",
-                                       "application/octet-stream");
+const core::Property InvokeHTTP::ContentType("Content-type",
+    "The Content-Type to specify for when content is being transmitted through a PUT, "
+    "POST or PATCH. In the case of an empty value after evaluating an expression language expression, "
+    "Content-Type defaults to",
+    "application/octet-stream");
 const core::Property InvokeHTTP::SendBody(
     core::PropertyBuilder::createProperty("send-message-body", "Send Body")
-      ->withDescription("DEPRECATED. Only kept for backwards compatibility, no functionality is included.")
-      ->withDefaultValue<bool>(true)
-      ->build());
+        ->withDescription("DEPRECATED. Only kept for backwards compatibility, no functionality is included.")
+        ->withDefaultValue<bool>(true)
+        ->build());
 const core::Property InvokeHTTP::SendMessageBody(
     core::PropertyBuilder::createProperty("Send Message Body")
-      ->withDescription("If true, sends the HTTP message body on POST/PUT/PATCH requests (default). "
-                        "If false, suppresses the message body and content-type header for these requests.")
-      ->withDefaultValue<bool>(true)
-      ->build());
-const core::Property InvokeHTTP::UseChunkedEncoding("Use Chunked Encoding", "When POST'ing, PUT'ing or PATCH'ing content set this property to true in order to not pass the 'Content-length' header"
-                                              " and instead send 'Transfer-Encoding' with a value of 'chunked'. This will enable the data transfer mechanism which was introduced in HTTP 1.1 "
-                                              "to pass data of unknown lengths in chunks.",
-                                              "false");
-const core::Property InvokeHTTP::PropPutOutputAttributes("Put Response Body in Attribute", "If set, the response body received back will be put into an attribute of the original "
-                                                   "FlowFile instead of a separate FlowFile. The attribute key to put to is determined by evaluating value of this property. ",
-                                                   "");
-const core::Property InvokeHTTP::AlwaysOutputResponse("Always Output Response", "Will force a response FlowFile to be generated and routed to the 'Response' relationship "
-                                                "regardless of what the server status code received is ",
-                                                "false");
-const core::Property InvokeHTTP::PenalizeOnNoRetry("Penalize on \"No Retry\"", "Enabling this property will penalize FlowFiles that are routed to the \"No Retry\" relationship.", "false");
+        ->withDescription("If true, sends the HTTP message body on POST/PUT/PATCH requests (default). "
+                          "If false, suppresses the message body and content-type header for these requests.")
+        ->withDefaultValue<bool>(true)
+        ->build());
+const core::Property InvokeHTTP::UseChunkedEncoding("Use Chunked Encoding",
+    "When POST'ing, PUT'ing or PATCH'ing content set this property to true in order to not pass the 'Content-length' header"
+    " and instead send 'Transfer-Encoding' with a value of 'chunked'."
+    " This will enable the data transfer mechanism which was introduced in HTTP 1.1 to pass data of unknown lengths in chunks.",
+    "false");
+const core::Property InvokeHTTP::PutResponseBodyInAttribute("Put Response Body in Attribute",
+    "If set, the response body received back will be put into an attribute of the original "
+    "FlowFile instead of a separate FlowFile. "
+    "The attribute key to put to is determined by evaluating value of this property. ",
+    "");
+const core::Property InvokeHTTP::AlwaysOutputResponse("Always Output Response",
+    "Will force a response FlowFile to be generated and routed to the 'Response' relationship regardless of what the server status code received is ",
+    "false");
+const core::Property InvokeHTTP::PenalizeOnNoRetry("Penalize on \"No Retry\"",
+    "Enabling this property will penalize FlowFiles that are routed to the \"No Retry\" relationship.",
+    "false");
 
 const core::Property InvokeHTTP::DisablePeerVerification("Disable Peer Verification", "Disables peer verification for the SSL session", "false");
 
 const core::Property InvokeHTTP::InvalidHTTPHeaderFieldHandlingStrategy(
     core::PropertyBuilder::createProperty("Invalid HTTP Header Field Handling Strategy")
-      ->withDescription("Indicates what should happen when an attribute's name is not a valid HTTP header field name. "
-        "Options: transform - invalid characters are replaced, fail - flow file is transferred to failure, drop - drops invalid attributes from HTTP message")
-      ->isRequired(true)
-      ->withDefaultValue<std::string>(toString(InvalidHTTPHeaderFieldHandlingOption::TRANSFORM))
-      ->withAllowableValues<std::string>(InvalidHTTPHeaderFieldHandlingOption::values())
-      ->build());
+        ->withDescription("Indicates what should happen when an attribute's name is not a valid HTTP header field name. "
+                          "Options: transform - invalid characters are replaced, fail - flow file is transferred to failure, drop - drops invalid attributes from HTTP message")
+        ->isRequired(true)
+        ->withDefaultValue<std::string>(toString(InvalidHTTPHeaderFieldHandlingOption::TRANSFORM))
+        ->withAllowableValues<std::string>(InvalidHTTPHeaderFieldHandlingOption::values())
+        ->build());
 
 
-const core::Relationship InvokeHTTP::Success("success", "The original FlowFile will be routed upon success (2xx status codes). "
-                                       "It will have new attributes detailing the success of the request.");
+const core::Relationship InvokeHTTP::Success("success",
+    "The original FlowFile will be routed upon success (2xx status codes). It will have new attributes detailing the success of the request.");
 
-const core::Relationship InvokeHTTP::RelResponse("response", "A Response FlowFile will be routed upon success (2xx status codes). "
-                                           "If the 'Always Output Response' property is true then the response will be sent "
-                                           "to this relationship regardless of the status code received.");
+const core::Relationship InvokeHTTP::RelResponse("response",
+    "A Response FlowFile will be routed upon success (2xx status codes). "
+    "If the 'Always Output Response' property is true then the response will be sent "
+    "to this relationship regardless of the status code received.");
 
-const core::Relationship InvokeHTTP::RelRetry("retry", "The original FlowFile will be routed on any status code that can be retried "
-                                        "(5xx status codes). It will have new attributes detailing the request.");
+const core::Relationship InvokeHTTP::RelRetry("retry",
+    "The original FlowFile will be routed on any status code that can be retried "
+    "(5xx status codes). It will have new attributes detailing the request.");
 
-const core::Relationship InvokeHTTP::RelNoRetry("no retry", "The original FlowFile will be routed on any status code that should NOT "
-                                          "be retried (1xx, 3xx, 4xx status codes). It will have new attributes detailing the request.");
+const core::Relationship InvokeHTTP::RelNoRetry("no retry",
+    "The original FlowFile will be routed on any status code that should NOT "
+    "be retried (1xx, 3xx, 4xx status codes). It will have new attributes detailing the request.");
 
-const core::Relationship InvokeHTTP::RelFailure("failure", "The original FlowFile will be routed on any type of connection failure, "
-                                          "timeout or general exception. It will have new attributes detailing the request.");
+const core::Relationship InvokeHTTP::RelFailure("failure",
+    "The original FlowFile will be routed on any type of connection failure, "
+    "timeout or general exception. It will have new attributes detailing the request.");
 
 void InvokeHTTP::initialize() {
   logger_->log_trace("Initializing InvokeHTTP");
   setSupportedProperties(properties());
   setSupportedRelationships(relationships());
 }
 
-void InvokeHTTP::onSchedule(const std::shared_ptr<core::ProcessContext> &context, const std::shared_ptr<core::ProcessSessionFactory>& /*sessionFactory*/) {
-  if (!context->getProperty(Method.getName(), method_)) {
-    logger_->log_debug("%s attribute is missing, so default value of %s will be used", Method.getName(), Method.getValue());
-    return;
-  }
+namespace {
+void setupClientTimeouts(extensions::curl::HTTPClient& client, const core::ProcessContext& context) {
+  if (auto connection_timeout = context.getProperty<core::TimePeriodValue>(InvokeHTTP::ConnectTimeout))
+    client.setConnectionTimeout(connection_timeout->getMilliseconds());
 
-  if (!context->getProperty(URL.getName(), url_)) {
-    logger_->log_debug("%s attribute is missing, so default value of %s will be used", URL.getName(), URL.getValue());
-    return;
-  }
+  if (auto read_timeout = context.getProperty<core::TimePeriodValue>(InvokeHTTP::ReadTimeout))
+    client.setReadTimeout(read_timeout->getMilliseconds());
+}
 
-  if (auto connect_timeout = context->getProperty<core::TimePeriodValue>(ConnectTimeout)) {
-    connect_timeout_ms_ =  connect_timeout->getMilliseconds();
-  } else {
-    logger_->log_debug("%s attribute is missing, so default value of %s will be used", ConnectTimeout.getName(), ConnectTimeout.getValue());
-    return;
+void setupClientProxy(extensions::curl::HTTPClient& client, const core::ProcessContext& context) {
+  utils::HTTPProxy proxy = {};
+  context.getProperty(InvokeHTTP::ProxyHost.getName(), proxy.host);
+  std::string port_str;
+  if (context.getProperty(InvokeHTTP::ProxyPort.getName(), port_str) && !port_str.empty()) {
+    core::Property::StringToInt(port_str, proxy.port);
   }
+  context.getProperty(InvokeHTTP::ProxyUsername.getName(), proxy.username);
+  context.getProperty(InvokeHTTP::ProxyPassword.getName(), proxy.password);
 
-  std::string content_type_str;
-  if (context->getProperty(ContentType.getName(), content_type_str)) {
-    content_type_ = content_type_str;
-  }
+  client.setHTTPProxy(proxy);
+}
 
-  if (auto read_timeout = context->getProperty<core::TimePeriodValue>(ReadTimeout)) {
-    read_timeout_ms_ =  read_timeout->getMilliseconds();
-  } else {
-    logger_->log_debug("%s attribute is missing, so default value of %s will be used", ReadTimeout.getName(), ReadTimeout.getValue());
-  }
+void setupClientPeerVerification(extensions::curl::HTTPClient& client, const core::ProcessContext& context) {
+  if (auto disable_peer_verification = context.getProperty<bool>(InvokeHTTP::DisablePeerVerification))
+    client.setPeerVerification(*disable_peer_verification);
+}
+
+void setupClientFollowRedirects(extensions::curl::HTTPClient& client, const core::ProcessContext& context) {
+  if (auto follow_redirects = context.getProperty<bool>(InvokeHTTP::FollowRedirects))
+    client.setFollowRedirects(*follow_redirects);
+}
 
-  std::string date_header_str;
-  if (!context->getProperty(DateHeader.getName(), date_header_str)) {
-    logger_->log_debug("%s attribute is missing, so default value of %s will be used", DateHeader.getName(), DateHeader.getValue());
+void setupClientContentType(extensions::curl::HTTPClient& client, const core::ProcessContext& context, bool send_body) {
+  if (auto content_type = context.getProperty(InvokeHTTP::ContentType)) {
+    if (send_body)
+      client.setContentType(*content_type);
   }
+}
 
-  date_header_include_ = utils::StringUtils::toBool(date_header_str).value_or(DateHeader.getValue());
+void setupClientTransferEncoding(extensions::curl::HTTPClient& client, bool use_chunked_encoding) {
+  if (use_chunked_encoding)
+    client.setRequestHeader("Transfer-Encoding", "chunked");

Review Comment:
   This is probably not a new bug, but the `DateHeader` and `UseChunkedEncoding` properties are never read.
   
   I'm surprised clang-tidy doesn't catch this in CI, because the clang-tidy plugin in my IDE flags this line as unreachable code.



##########
extensions/http-curl/processors/InvokeHTTP.cpp:
##########
@@ -83,165 +84,188 @@ const core::Property InvokeHTTP::ProxyUsername(
     core::PropertyBuilder::createProperty("invokehttp-proxy-username", "Proxy Username")->withDescription("Username to set when authenticating against proxy")->isRequired(false)->build());
 const core::Property InvokeHTTP::ProxyPassword(
     core::PropertyBuilder::createProperty("invokehttp-proxy-password", "Proxy Password")->withDescription("Password to set when authenticating against proxy")->isRequired(false)->build());
-const core::Property InvokeHTTP::ContentType("Content-type", "The Content-Type to specify for when content is being transmitted through a PUT, "
-                                       "POST or PATCH. In the case of an empty value after evaluating an expression language expression, "
-                                       "Content-Type defaults to",
-                                       "application/octet-stream");
+const core::Property InvokeHTTP::ContentType("Content-type",
+    "The Content-Type to specify for when content is being transmitted through a PUT, "
+    "POST or PATCH. In the case of an empty value after evaluating an expression language expression, "
+    "Content-Type defaults to",
+    "application/octet-stream");
 const core::Property InvokeHTTP::SendBody(
     core::PropertyBuilder::createProperty("send-message-body", "Send Body")
-      ->withDescription("DEPRECATED. Only kept for backwards compatibility, no functionality is included.")
-      ->withDefaultValue<bool>(true)
-      ->build());
+        ->withDescription("DEPRECATED. Only kept for backwards compatibility, no functionality is included.")
+        ->withDefaultValue<bool>(true)
+        ->build());
 const core::Property InvokeHTTP::SendMessageBody(
     core::PropertyBuilder::createProperty("Send Message Body")
-      ->withDescription("If true, sends the HTTP message body on POST/PUT/PATCH requests (default). "
-                        "If false, suppresses the message body and content-type header for these requests.")
-      ->withDefaultValue<bool>(true)
-      ->build());
-const core::Property InvokeHTTP::UseChunkedEncoding("Use Chunked Encoding", "When POST'ing, PUT'ing or PATCH'ing content set this property to true in order to not pass the 'Content-length' header"
-                                              " and instead send 'Transfer-Encoding' with a value of 'chunked'. This will enable the data transfer mechanism which was introduced in HTTP 1.1 "
-                                              "to pass data of unknown lengths in chunks.",
-                                              "false");
-const core::Property InvokeHTTP::PropPutOutputAttributes("Put Response Body in Attribute", "If set, the response body received back will be put into an attribute of the original "
-                                                   "FlowFile instead of a separate FlowFile. The attribute key to put to is determined by evaluating value of this property. ",
-                                                   "");
-const core::Property InvokeHTTP::AlwaysOutputResponse("Always Output Response", "Will force a response FlowFile to be generated and routed to the 'Response' relationship "
-                                                "regardless of what the server status code received is ",
-                                                "false");
-const core::Property InvokeHTTP::PenalizeOnNoRetry("Penalize on \"No Retry\"", "Enabling this property will penalize FlowFiles that are routed to the \"No Retry\" relationship.", "false");
+        ->withDescription("If true, sends the HTTP message body on POST/PUT/PATCH requests (default). "
+                          "If false, suppresses the message body and content-type header for these requests.")
+        ->withDefaultValue<bool>(true)
+        ->build());
+const core::Property InvokeHTTP::UseChunkedEncoding("Use Chunked Encoding",
+    "When POST'ing, PUT'ing or PATCH'ing content set this property to true in order to not pass the 'Content-length' header"
+    " and instead send 'Transfer-Encoding' with a value of 'chunked'."
+    " This will enable the data transfer mechanism which was introduced in HTTP 1.1 to pass data of unknown lengths in chunks.",
+    "false");
+const core::Property InvokeHTTP::PutResponseBodyInAttribute("Put Response Body in Attribute",
+    "If set, the response body received back will be put into an attribute of the original "
+    "FlowFile instead of a separate FlowFile. "
+    "The attribute key to put to is determined by evaluating value of this property. ",
+    "");
+const core::Property InvokeHTTP::AlwaysOutputResponse("Always Output Response",
+    "Will force a response FlowFile to be generated and routed to the 'Response' relationship regardless of what the server status code received is ",
+    "false");
+const core::Property InvokeHTTP::PenalizeOnNoRetry("Penalize on \"No Retry\"",
+    "Enabling this property will penalize FlowFiles that are routed to the \"No Retry\" relationship.",
+    "false");
 
 const core::Property InvokeHTTP::DisablePeerVerification("Disable Peer Verification", "Disables peer verification for the SSL session", "false");
 
 const core::Property InvokeHTTP::InvalidHTTPHeaderFieldHandlingStrategy(
     core::PropertyBuilder::createProperty("Invalid HTTP Header Field Handling Strategy")
-      ->withDescription("Indicates what should happen when an attribute's name is not a valid HTTP header field name. "
-        "Options: transform - invalid characters are replaced, fail - flow file is transferred to failure, drop - drops invalid attributes from HTTP message")
-      ->isRequired(true)
-      ->withDefaultValue<std::string>(toString(InvalidHTTPHeaderFieldHandlingOption::TRANSFORM))
-      ->withAllowableValues<std::string>(InvalidHTTPHeaderFieldHandlingOption::values())
-      ->build());
+        ->withDescription("Indicates what should happen when an attribute's name is not a valid HTTP header field name. "
+                          "Options: transform - invalid characters are replaced, fail - flow file is transferred to failure, drop - drops invalid attributes from HTTP message")
+        ->isRequired(true)
+        ->withDefaultValue<std::string>(toString(InvalidHTTPHeaderFieldHandlingOption::TRANSFORM))
+        ->withAllowableValues<std::string>(InvalidHTTPHeaderFieldHandlingOption::values())
+        ->build());
 
 
-const core::Relationship InvokeHTTP::Success("success", "The original FlowFile will be routed upon success (2xx status codes). "
-                                       "It will have new attributes detailing the success of the request.");
+const core::Relationship InvokeHTTP::Success("success",
+    "The original FlowFile will be routed upon success (2xx status codes). It will have new attributes detailing the success of the request.");
 
-const core::Relationship InvokeHTTP::RelResponse("response", "A Response FlowFile will be routed upon success (2xx status codes). "
-                                           "If the 'Always Output Response' property is true then the response will be sent "
-                                           "to this relationship regardless of the status code received.");
+const core::Relationship InvokeHTTP::RelResponse("response",
+    "A Response FlowFile will be routed upon success (2xx status codes). "
+    "If the 'Always Output Response' property is true then the response will be sent "
+    "to this relationship regardless of the status code received.");
 
-const core::Relationship InvokeHTTP::RelRetry("retry", "The original FlowFile will be routed on any status code that can be retried "
-                                        "(5xx status codes). It will have new attributes detailing the request.");
+const core::Relationship InvokeHTTP::RelRetry("retry",
+    "The original FlowFile will be routed on any status code that can be retried "
+    "(5xx status codes). It will have new attributes detailing the request.");
 
-const core::Relationship InvokeHTTP::RelNoRetry("no retry", "The original FlowFile will be routed on any status code that should NOT "
-                                          "be retried (1xx, 3xx, 4xx status codes). It will have new attributes detailing the request.");
+const core::Relationship InvokeHTTP::RelNoRetry("no retry",
+    "The original FlowFile will be routed on any status code that should NOT "
+    "be retried (1xx, 3xx, 4xx status codes). It will have new attributes detailing the request.");
 
-const core::Relationship InvokeHTTP::RelFailure("failure", "The original FlowFile will be routed on any type of connection failure, "
-                                          "timeout or general exception. It will have new attributes detailing the request.");
+const core::Relationship InvokeHTTP::RelFailure("failure",
+    "The original FlowFile will be routed on any type of connection failure, "
+    "timeout or general exception. It will have new attributes detailing the request.");
 
 void InvokeHTTP::initialize() {
   logger_->log_trace("Initializing InvokeHTTP");
   setSupportedProperties(properties());
   setSupportedRelationships(relationships());
 }
 
-void InvokeHTTP::onSchedule(const std::shared_ptr<core::ProcessContext> &context, const std::shared_ptr<core::ProcessSessionFactory>& /*sessionFactory*/) {
-  if (!context->getProperty(Method.getName(), method_)) {
-    logger_->log_debug("%s attribute is missing, so default value of %s will be used", Method.getName(), Method.getValue());
-    return;
-  }
+namespace {
+void setupClientTimeouts(extensions::curl::HTTPClient& client, const core::ProcessContext& context) {
+  if (auto connection_timeout = context.getProperty<core::TimePeriodValue>(InvokeHTTP::ConnectTimeout))
+    client.setConnectionTimeout(connection_timeout->getMilliseconds());
 
-  if (!context->getProperty(URL.getName(), url_)) {
-    logger_->log_debug("%s attribute is missing, so default value of %s will be used", URL.getName(), URL.getValue());
-    return;
-  }
+  if (auto read_timeout = context.getProperty<core::TimePeriodValue>(InvokeHTTP::ReadTimeout))
+    client.setReadTimeout(read_timeout->getMilliseconds());
+}
 
-  if (auto connect_timeout = context->getProperty<core::TimePeriodValue>(ConnectTimeout)) {
-    connect_timeout_ms_ =  connect_timeout->getMilliseconds();
-  } else {
-    logger_->log_debug("%s attribute is missing, so default value of %s will be used", ConnectTimeout.getName(), ConnectTimeout.getValue());
-    return;
+void setupClientProxy(extensions::curl::HTTPClient& client, const core::ProcessContext& context) {
+  utils::HTTPProxy proxy = {};
+  context.getProperty(InvokeHTTP::ProxyHost.getName(), proxy.host);
+  std::string port_str;
+  if (context.getProperty(InvokeHTTP::ProxyPort.getName(), port_str) && !port_str.empty()) {
+    core::Property::StringToInt(port_str, proxy.port);
   }
+  context.getProperty(InvokeHTTP::ProxyUsername.getName(), proxy.username);
+  context.getProperty(InvokeHTTP::ProxyPassword.getName(), proxy.password);
 
-  std::string content_type_str;
-  if (context->getProperty(ContentType.getName(), content_type_str)) {
-    content_type_ = content_type_str;
-  }
+  client.setHTTPProxy(proxy);
+}
 
-  if (auto read_timeout = context->getProperty<core::TimePeriodValue>(ReadTimeout)) {
-    read_timeout_ms_ =  read_timeout->getMilliseconds();
-  } else {
-    logger_->log_debug("%s attribute is missing, so default value of %s will be used", ReadTimeout.getName(), ReadTimeout.getValue());
-  }
+void setupClientPeerVerification(extensions::curl::HTTPClient& client, const core::ProcessContext& context) {
+  if (auto disable_peer_verification = context.getProperty<bool>(InvokeHTTP::DisablePeerVerification))
+    client.setPeerVerification(*disable_peer_verification);
+}
+
+void setupClientFollowRedirects(extensions::curl::HTTPClient& client, const core::ProcessContext& context) {
+  if (auto follow_redirects = context.getProperty<bool>(InvokeHTTP::FollowRedirects))
+    client.setFollowRedirects(*follow_redirects);
+}
 
-  std::string date_header_str;
-  if (!context->getProperty(DateHeader.getName(), date_header_str)) {
-    logger_->log_debug("%s attribute is missing, so default value of %s will be used", DateHeader.getName(), DateHeader.getValue());
+void setupClientContentType(extensions::curl::HTTPClient& client, const core::ProcessContext& context, bool send_body) {
+  if (auto content_type = context.getProperty(InvokeHTTP::ContentType)) {
+    if (send_body)
+      client.setContentType(*content_type);
   }
+}
 
-  date_header_include_ = utils::StringUtils::toBool(date_header_str).value_or(DateHeader.getValue());
+void setupClientTransferEncoding(extensions::curl::HTTPClient& client, bool use_chunked_encoding) {
+  if (use_chunked_encoding)
+    client.setRequestHeader("Transfer-Encoding", "chunked");
+  else
+    client.setRequestHeader("Transfer-Encoding", std::nullopt);
+}
+}  // namespace
 
-  if (!context->getProperty(PropPutOutputAttributes.getName(), put_attribute_name_)) {
-    logger_->log_debug("%s attribute is missing, so default value of %s will be used", PropPutOutputAttributes.getName(), PropPutOutputAttributes.getValue());
-  }
+void InvokeHTTP::setupMembersFromProperties(const core::ProcessContext& context) {
+  context.getProperty(SendMessageBody.getName(), send_body_);
 
-  attributes_to_send_ = context->getProperty(AttributesToSend)
-      | utils::filter([](const std::string& s) { return !s.empty(); })  // avoid compiling an empty string to regex
-      | utils::map([](const std::string& regex_str) { return utils::Regex{regex_str}; })
-      | utils::orElse([this] { logger_->log_debug("%s is missing, so the default value will be used", AttributesToSend.getName()); });
+  attributes_to_send_ = context.getProperty(AttributesToSend)
+                        | utils::filter([](const std::string& s) { return !s.empty(); })  // avoid compiling an empty string to regex
+                        | utils::map([](const std::string& regex_str) { return utils::Regex{regex_str}; })
+                        | utils::orElse([this] { logger_->log_debug("%s is missing, so the default value will be used", AttributesToSend.getName()); });
 
-  std::string always_output_response;
-  if (!context->getProperty(AlwaysOutputResponse.getName(), always_output_response)) {
-    logger_->log_debug("%s attribute is missing, so default value of %s will be used", AlwaysOutputResponse.getName(), AlwaysOutputResponse.getValue());
-  }
 
-  always_output_response_ = utils::StringUtils::toBool(always_output_response).value_or(false);
+  always_output_response_ = context.getProperty<bool>(AlwaysOutputResponse).value_or(false);
+  penalize_no_retry_ = context.getProperty<bool>(PenalizeOnNoRetry).value_or(false);
 
-  std::string penalize_no_retry = "false";
-  if (!context->getProperty(PenalizeOnNoRetry.getName(), penalize_no_retry)) {
-    logger_->log_debug("%s attribute is missing, so default value of %s will be used", PenalizeOnNoRetry.getName(), PenalizeOnNoRetry.getValue());
-  }
+  invalid_http_header_field_handling_strategy_ = utils::parseEnumProperty<InvalidHTTPHeaderFieldHandlingOption>(context, InvalidHTTPHeaderFieldHandlingStrategy);
 
-  penalize_no_retry_ = utils::StringUtils::toBool(penalize_no_retry).value_or(false);
+  put_response_body_in_attribute_ = context.getProperty(PutResponseBodyInAttribute);
+  if (put_response_body_in_attribute_ && put_response_body_in_attribute_->empty()) {
+    logger_->log_warn("%s is set to an empty string", PutResponseBodyInAttribute.getName());
+    put_response_body_in_attribute_.reset();
+  }
+}
 
-  std::string context_name;
-  if (context->getProperty(SSLContext.getName(), context_name) && !IsNullOrEmpty(context_name)) {
-    std::shared_ptr<core::controller::ControllerService> service = context->getControllerService(context_name);
-    if (!service) {
-      logger_->log_error("Couldn't find controller service with name '%s'", context_name);
+std::unique_ptr<minifi::extensions::curl::HTTPClient> InvokeHTTP::createHTTPClientFromPropertiesAndMembers(const core::ProcessContext& context) const {
+  std::string method;
+  if (!context.getProperty(Method.getName(), method))
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Method property missing or invalid");
+
+  std::string url;
+  if (!context.getProperty(URL.getName(), url))
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "URL property missing or invalid");
+
+  std::shared_ptr<minifi::controllers::SSLContextService> ssl_context_service;
+  if (auto ssl_context_name = context.getProperty(SSLContext)) {
+    if (auto service = context.getControllerService(*ssl_context_name)) {
+      ssl_context_service = std::dynamic_pointer_cast<minifi::controllers::SSLContextService>(service);
+      if (!ssl_context_service)
+        logger_->log_error("Controller service '%s' is not an SSLContextService", *ssl_context_name);
     } else {
-      ssl_context_service_ = std::dynamic_pointer_cast<minifi::controllers::SSLContextService>(service);
-      if (!ssl_context_service_) {
-        logger_->log_error("Controller service '%s' is not an SSLContextService", context_name);
-      }
+      logger_->log_error("Couldn't find controller service with name '%s'", *ssl_context_name);
     }
   }
 
-  std::string use_chunked_encoding = "false";
-  if (!context->getProperty(UseChunkedEncoding.getName(), use_chunked_encoding)) {
-    logger_->log_debug("%s attribute is missing, so default value of %s will be used", UseChunkedEncoding.getName(), UseChunkedEncoding.getValue());
-  }
+  auto client = std::make_unique<minifi::extensions::curl::HTTPClient>();
+  client->initialize(std::move(method), std::move(url), std::move(ssl_context_service));
+  setupClientTimeouts(*client, context);
+  setupClientProxy(*client, context);
+  setupClientFollowRedirects(*client, context);
+  setupClientPeerVerification(*client, context);
+  setupClientContentType(*client, context, send_body_);
+  setupClientTransferEncoding(*client, use_chunked_encoding_);
 
-  use_chunked_encoding_ = utils::StringUtils::toBool(use_chunked_encoding).value_or(false);
+  return client;
+}
 
-  std::string disable_peer_verification;
-  disable_peer_verification_ = (context->getProperty(DisablePeerVerification.getName(), disable_peer_verification) && utils::StringUtils::toBool(disable_peer_verification).value_or(false));
 
-  proxy_ = {};
-  context->getProperty(ProxyHost.getName(), proxy_.host);
-  std::string port_str;
-  if (context->getProperty(ProxyPort.getName(), port_str) && !port_str.empty()) {
-    core::Property::StringToInt(port_str, proxy_.port);
-  }
-  context->getProperty(ProxyUsername.getName(), proxy_.username);
-  context->getProperty(ProxyPassword.getName(), proxy_.password);
-  context->getProperty(FollowRedirects.getName(), follow_redirects_);
-  context->getProperty(SendMessageBody.getName(), send_body_);
+void InvokeHTTP::onSchedule(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSessionFactory>& /*sessionFactory*/) {
+  gsl_Expects(context);
 
-  invalid_http_header_field_handling_strategy_ = utils::parseEnumProperty<InvalidHTTPHeaderFieldHandlingOption>(*context, InvalidHTTPHeaderFieldHandlingStrategy);
+  setupMembersFromProperties(*context);
+  client_queue_ = gsl::make_not_null(utils::ResourceQueue<extensions::curl::HTTPClient>::create(getMaxConcurrentTasks(), logger_));
 }
 
-bool InvokeHTTP::shouldEmitFlowFile() const {
-  return ("POST" == method_ || "PUT" == method_ || "PATCH" == method_);
+bool InvokeHTTP::shouldEmitFlowFile(minifi::extensions::curl::HTTPClient& client) {

Review Comment:
   why is this function no longer const?



##########
extensions/http-curl/processors/InvokeHTTP.cpp:
##########
@@ -83,165 +84,188 @@ const core::Property InvokeHTTP::ProxyUsername(
     core::PropertyBuilder::createProperty("invokehttp-proxy-username", "Proxy Username")->withDescription("Username to set when authenticating against proxy")->isRequired(false)->build());
 const core::Property InvokeHTTP::ProxyPassword(
     core::PropertyBuilder::createProperty("invokehttp-proxy-password", "Proxy Password")->withDescription("Password to set when authenticating against proxy")->isRequired(false)->build());
-const core::Property InvokeHTTP::ContentType("Content-type", "The Content-Type to specify for when content is being transmitted through a PUT, "
-                                       "POST or PATCH. In the case of an empty value after evaluating an expression language expression, "
-                                       "Content-Type defaults to",
-                                       "application/octet-stream");
+const core::Property InvokeHTTP::ContentType("Content-type",
+    "The Content-Type to specify for when content is being transmitted through a PUT, "
+    "POST or PATCH. In the case of an empty value after evaluating an expression language expression, "
+    "Content-Type defaults to",
+    "application/octet-stream");
 const core::Property InvokeHTTP::SendBody(
     core::PropertyBuilder::createProperty("send-message-body", "Send Body")
-      ->withDescription("DEPRECATED. Only kept for backwards compatibility, no functionality is included.")
-      ->withDefaultValue<bool>(true)
-      ->build());
+        ->withDescription("DEPRECATED. Only kept for backwards compatibility, no functionality is included.")
+        ->withDefaultValue<bool>(true)
+        ->build());
 const core::Property InvokeHTTP::SendMessageBody(
     core::PropertyBuilder::createProperty("Send Message Body")
-      ->withDescription("If true, sends the HTTP message body on POST/PUT/PATCH requests (default). "
-                        "If false, suppresses the message body and content-type header for these requests.")
-      ->withDefaultValue<bool>(true)
-      ->build());
-const core::Property InvokeHTTP::UseChunkedEncoding("Use Chunked Encoding", "When POST'ing, PUT'ing or PATCH'ing content set this property to true in order to not pass the 'Content-length' header"
-                                              " and instead send 'Transfer-Encoding' with a value of 'chunked'. This will enable the data transfer mechanism which was introduced in HTTP 1.1 "
-                                              "to pass data of unknown lengths in chunks.",
-                                              "false");
-const core::Property InvokeHTTP::PropPutOutputAttributes("Put Response Body in Attribute", "If set, the response body received back will be put into an attribute of the original "
-                                                   "FlowFile instead of a separate FlowFile. The attribute key to put to is determined by evaluating value of this property. ",
-                                                   "");
-const core::Property InvokeHTTP::AlwaysOutputResponse("Always Output Response", "Will force a response FlowFile to be generated and routed to the 'Response' relationship "
-                                                "regardless of what the server status code received is ",
-                                                "false");
-const core::Property InvokeHTTP::PenalizeOnNoRetry("Penalize on \"No Retry\"", "Enabling this property will penalize FlowFiles that are routed to the \"No Retry\" relationship.", "false");
+        ->withDescription("If true, sends the HTTP message body on POST/PUT/PATCH requests (default). "
+                          "If false, suppresses the message body and content-type header for these requests.")
+        ->withDefaultValue<bool>(true)
+        ->build());
+const core::Property InvokeHTTP::UseChunkedEncoding("Use Chunked Encoding",
+    "When POST'ing, PUT'ing or PATCH'ing content set this property to true in order to not pass the 'Content-length' header"
+    " and instead send 'Transfer-Encoding' with a value of 'chunked'."
+    " This will enable the data transfer mechanism which was introduced in HTTP 1.1 to pass data of unknown lengths in chunks.",
+    "false");
+const core::Property InvokeHTTP::PutResponseBodyInAttribute("Put Response Body in Attribute",
+    "If set, the response body received back will be put into an attribute of the original "
+    "FlowFile instead of a separate FlowFile. "
+    "The attribute key to put to is determined by evaluating value of this property. ",
+    "");
+const core::Property InvokeHTTP::AlwaysOutputResponse("Always Output Response",
+    "Will force a response FlowFile to be generated and routed to the 'Response' relationship regardless of what the server status code received is ",
+    "false");
+const core::Property InvokeHTTP::PenalizeOnNoRetry("Penalize on \"No Retry\"",
+    "Enabling this property will penalize FlowFiles that are routed to the \"No Retry\" relationship.",
+    "false");
 
 const core::Property InvokeHTTP::DisablePeerVerification("Disable Peer Verification", "Disables peer verification for the SSL session", "false");
 
 const core::Property InvokeHTTP::InvalidHTTPHeaderFieldHandlingStrategy(
     core::PropertyBuilder::createProperty("Invalid HTTP Header Field Handling Strategy")
-      ->withDescription("Indicates what should happen when an attribute's name is not a valid HTTP header field name. "
-        "Options: transform - invalid characters are replaced, fail - flow file is transferred to failure, drop - drops invalid attributes from HTTP message")
-      ->isRequired(true)
-      ->withDefaultValue<std::string>(toString(InvalidHTTPHeaderFieldHandlingOption::TRANSFORM))
-      ->withAllowableValues<std::string>(InvalidHTTPHeaderFieldHandlingOption::values())
-      ->build());
+        ->withDescription("Indicates what should happen when an attribute's name is not a valid HTTP header field name. "
+                          "Options: transform - invalid characters are replaced, fail - flow file is transferred to failure, drop - drops invalid attributes from HTTP message")
+        ->isRequired(true)
+        ->withDefaultValue<std::string>(toString(InvalidHTTPHeaderFieldHandlingOption::TRANSFORM))
+        ->withAllowableValues<std::string>(InvalidHTTPHeaderFieldHandlingOption::values())
+        ->build());
 
 
-const core::Relationship InvokeHTTP::Success("success", "The original FlowFile will be routed upon success (2xx status codes). "
-                                       "It will have new attributes detailing the success of the request.");
+const core::Relationship InvokeHTTP::Success("success",
+    "The original FlowFile will be routed upon success (2xx status codes). It will have new attributes detailing the success of the request.");
 
-const core::Relationship InvokeHTTP::RelResponse("response", "A Response FlowFile will be routed upon success (2xx status codes). "
-                                           "If the 'Always Output Response' property is true then the response will be sent "
-                                           "to this relationship regardless of the status code received.");
+const core::Relationship InvokeHTTP::RelResponse("response",
+    "A Response FlowFile will be routed upon success (2xx status codes). "
+    "If the 'Always Output Response' property is true then the response will be sent "
+    "to this relationship regardless of the status code received.");
 
-const core::Relationship InvokeHTTP::RelRetry("retry", "The original FlowFile will be routed on any status code that can be retried "
-                                        "(5xx status codes). It will have new attributes detailing the request.");
+const core::Relationship InvokeHTTP::RelRetry("retry",
+    "The original FlowFile will be routed on any status code that can be retried "
+    "(5xx status codes). It will have new attributes detailing the request.");
 
-const core::Relationship InvokeHTTP::RelNoRetry("no retry", "The original FlowFile will be routed on any status code that should NOT "
-                                          "be retried (1xx, 3xx, 4xx status codes). It will have new attributes detailing the request.");
+const core::Relationship InvokeHTTP::RelNoRetry("no retry",
+    "The original FlowFile will be routed on any status code that should NOT "
+    "be retried (1xx, 3xx, 4xx status codes). It will have new attributes detailing the request.");
 
-const core::Relationship InvokeHTTP::RelFailure("failure", "The original FlowFile will be routed on any type of connection failure, "
-                                          "timeout or general exception. It will have new attributes detailing the request.");
+const core::Relationship InvokeHTTP::RelFailure("failure",
+    "The original FlowFile will be routed on any type of connection failure, "
+    "timeout or general exception. It will have new attributes detailing the request.");
 
 void InvokeHTTP::initialize() {
   logger_->log_trace("Initializing InvokeHTTP");
   setSupportedProperties(properties());
   setSupportedRelationships(relationships());
 }
 
-void InvokeHTTP::onSchedule(const std::shared_ptr<core::ProcessContext> &context, const std::shared_ptr<core::ProcessSessionFactory>& /*sessionFactory*/) {
-  if (!context->getProperty(Method.getName(), method_)) {
-    logger_->log_debug("%s attribute is missing, so default value of %s will be used", Method.getName(), Method.getValue());
-    return;
-  }
+namespace {
+void setupClientTimeouts(extensions::curl::HTTPClient& client, const core::ProcessContext& context) {
+  if (auto connection_timeout = context.getProperty<core::TimePeriodValue>(InvokeHTTP::ConnectTimeout))
+    client.setConnectionTimeout(connection_timeout->getMilliseconds());
 
-  if (!context->getProperty(URL.getName(), url_)) {
-    logger_->log_debug("%s attribute is missing, so default value of %s will be used", URL.getName(), URL.getValue());
-    return;
-  }
+  if (auto read_timeout = context.getProperty<core::TimePeriodValue>(InvokeHTTP::ReadTimeout))
+    client.setReadTimeout(read_timeout->getMilliseconds());
+}
 
-  if (auto connect_timeout = context->getProperty<core::TimePeriodValue>(ConnectTimeout)) {
-    connect_timeout_ms_ =  connect_timeout->getMilliseconds();
-  } else {
-    logger_->log_debug("%s attribute is missing, so default value of %s will be used", ConnectTimeout.getName(), ConnectTimeout.getValue());
-    return;
+void setupClientProxy(extensions::curl::HTTPClient& client, const core::ProcessContext& context) {
+  utils::HTTPProxy proxy = {};
+  context.getProperty(InvokeHTTP::ProxyHost.getName(), proxy.host);
+  std::string port_str;
+  if (context.getProperty(InvokeHTTP::ProxyPort.getName(), port_str) && !port_str.empty()) {
+    core::Property::StringToInt(port_str, proxy.port);
   }
+  context.getProperty(InvokeHTTP::ProxyUsername.getName(), proxy.username);
+  context.getProperty(InvokeHTTP::ProxyPassword.getName(), proxy.password);
 
-  std::string content_type_str;
-  if (context->getProperty(ContentType.getName(), content_type_str)) {
-    content_type_ = content_type_str;
-  }
+  client.setHTTPProxy(proxy);
+}
 
-  if (auto read_timeout = context->getProperty<core::TimePeriodValue>(ReadTimeout)) {
-    read_timeout_ms_ =  read_timeout->getMilliseconds();
-  } else {
-    logger_->log_debug("%s attribute is missing, so default value of %s will be used", ReadTimeout.getName(), ReadTimeout.getValue());
-  }
+void setupClientPeerVerification(extensions::curl::HTTPClient& client, const core::ProcessContext& context) {
+  if (auto disable_peer_verification = context.getProperty<bool>(InvokeHTTP::DisablePeerVerification))
+    client.setPeerVerification(*disable_peer_verification);
+}
+
+void setupClientFollowRedirects(extensions::curl::HTTPClient& client, const core::ProcessContext& context) {
+  if (auto follow_redirects = context.getProperty<bool>(InvokeHTTP::FollowRedirects))
+    client.setFollowRedirects(*follow_redirects);
+}
 
-  std::string date_header_str;
-  if (!context->getProperty(DateHeader.getName(), date_header_str)) {
-    logger_->log_debug("%s attribute is missing, so default value of %s will be used", DateHeader.getName(), DateHeader.getValue());
+void setupClientContentType(extensions::curl::HTTPClient& client, const core::ProcessContext& context, bool send_body) {
+  if (auto content_type = context.getProperty(InvokeHTTP::ContentType)) {
+    if (send_body)
+      client.setContentType(*content_type);
   }
+}
 
-  date_header_include_ = utils::StringUtils::toBool(date_header_str).value_or(DateHeader.getValue());
+void setupClientTransferEncoding(extensions::curl::HTTPClient& client, bool use_chunked_encoding) {
+  if (use_chunked_encoding)
+    client.setRequestHeader("Transfer-Encoding", "chunked");
+  else
+    client.setRequestHeader("Transfer-Encoding", std::nullopt);
+}
+}  // namespace
 
-  if (!context->getProperty(PropPutOutputAttributes.getName(), put_attribute_name_)) {
-    logger_->log_debug("%s attribute is missing, so default value of %s will be used", PropPutOutputAttributes.getName(), PropPutOutputAttributes.getValue());
-  }
+void InvokeHTTP::setupMembersFromProperties(const core::ProcessContext& context) {
+  context.getProperty(SendMessageBody.getName(), send_body_);

Review Comment:
   I would rename `send_body_` to `send_message_body_`, as there is also a (non-functional) `SendBody` property, which makes this confusing.



##########
extensions/http-curl/processors/InvokeHTTP.h:
##########
@@ -127,52 +130,37 @@ class InvokeHTTP : public core::Processor {
 
   EXTENSIONAPI static constexpr const char* STATUS_CODE = "invokehttp.status.code";
   EXTENSIONAPI static constexpr const char* STATUS_MESSAGE = "invokehttp.status.message";
-  EXTENSIONAPI static constexpr const char* RESPONSE_BODY = "invokehttp.response.body";
   EXTENSIONAPI static constexpr const char* REQUEST_URL = "invokehttp.request.url";
   EXTENSIONAPI static constexpr const char* TRANSACTION_ID = "invokehttp.tx.id";
-  EXTENSIONAPI static constexpr const char* REMOTE_DN = "invokehttp.remote.dn";
-  EXTENSIONAPI static constexpr const char* EXCEPTION_CLASS = "invokehttp.java.exception.class";
-  EXTENSIONAPI static constexpr const char* EXCEPTION_MESSAGE = "invokehttp.java.exception.message";
 
-  void onTrigger(const std::shared_ptr<core::ProcessContext> &context, const std::shared_ptr<core::ProcessSession> &session) override;
+  void onTrigger(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSession>& session) override;
   void initialize() override;
-  void onSchedule(const std::shared_ptr<core::ProcessContext> &context, const std::shared_ptr<core::ProcessSessionFactory> &sessionFactory) override;
+  void onSchedule(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSessionFactory>& sessionFactory) override;
 
  private:
-  /**
-   * Routes the flowfile to the proper destination
-   * @param request request flow file record
-   * @param response response flow file record
-   * @param session process session
-   * @param context process context
-   * @param isSuccess success code or not
-   * @param statuscode http response code.
-   */
-  void route(const std::shared_ptr<core::FlowFile> &request, const std::shared_ptr<core::FlowFile> &response, const std::shared_ptr<core::ProcessSession> &session,
-             const std::shared_ptr<core::ProcessContext> &context, bool is_success, int64_t status_code);
-  bool shouldEmitFlowFile() const;
+  void route(const std::shared_ptr<core::FlowFile>& request, const std::shared_ptr<core::FlowFile>& response, const std::shared_ptr<core::ProcessSession>& session,
+             const std::shared_ptr<core::ProcessContext>& context, bool is_success, int64_t status_code);
+  static bool shouldEmitFlowFile(minifi::extensions::curl::HTTPClient& client);
+  void onTriggerWithClient(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSession>& session, minifi::extensions::curl::HTTPClient& client);
   [[nodiscard]] bool appendHeaders(const core::FlowFile& flow_file, /*std::invocable<std::string, std::string>*/ auto append_header);
 
-  std::shared_ptr<minifi::controllers::SSLContextService> ssl_context_service_;
-  std::string method_;
-  std::string url_;
-  bool date_header_include_{true};
+
+  void setupMembersFromProperties(const core::ProcessContext& context);
+  std::unique_ptr<minifi::extensions::curl::HTTPClient> createHTTPClientFromPropertiesAndMembers(const core::ProcessContext& context) const;
+
   std::optional<utils::Regex> attributes_to_send_;
-  std::chrono::milliseconds connect_timeout_ms_{20000};
-  std::chrono::milliseconds read_timeout_ms_{20000};
-  // attribute in which response body will be added
-  std::string put_attribute_name_;
+
+  std::optional<std::string> put_response_body_in_attribute_;
   bool always_output_response_{false};
-  std::string content_type_;
   bool use_chunked_encoding_{false};
   bool penalize_no_retry_{false};
-  // disabling peer verification makes susceptible for MITM attacks
-  bool disable_peer_verification_{false};
-  utils::HTTPProxy proxy_;
-  bool follow_redirects_{true};
   bool send_body_{true};
+
   InvalidHTTPHeaderFieldHandlingOption invalid_http_header_field_handling_strategy_;
+
   std::shared_ptr<core::logging::Logger> logger_{core::logging::LoggerFactory<InvokeHTTP>::getLogger()};
+  gsl::not_null<std::shared_ptr<utils::ResourceQueue<extensions::curl::HTTPClient>>> client_queue_ = gsl::make_not_null(
+      utils::ResourceQueue<extensions::curl::HTTPClient>::create(getMaxConcurrentTasks(), logger_));

Review Comment:
   has `max_concurrent_tasks_` been already set when this runs?



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

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

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


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

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


##########
extensions/http-curl/client/RequestHeaders.h:
##########
@@ -0,0 +1,58 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#pragma once
+
+#include <unordered_map>
+#include <memory>
+#include <optional>
+#include <string>
+#include <string_view>
+
+#ifdef WIN32
+#pragma comment(lib, "wldap32.lib" )
+#pragma comment(lib, "crypt32.lib" )
+#pragma comment(lib, "Ws2_32.lib")
+
+#define CURL_STATICLIB
+#include <curl/curl.h>
+#else
+#include <curl/curl.h>
+#endif
+
+namespace org::apache::nifi::minifi::extensions::curl {
+class RequestHeaders {
+ public:
+  RequestHeaders() = default;
+
+  void appendHeader(std::string key, std::string value);
+
+  void disableExpectHeader();
+
+  [[nodiscard]] std::unique_ptr<struct curl_slist, decltype(&curl_slist_free_all)> get() const;
+  [[nodiscard]] auto size() const { return headers_.size(); }
+  [[nodiscard]] bool empty() const;
+
+  std::string& operator[](const std::string& key);
+  std::string& operator[](std::string&& key);
+
+  [[nodiscard]] bool contains(const std::string& key) const;
+  void erase(const std::string& key);
+
+ private:
+  std::unordered_map<std::string, std::string> headers_;
+};

Review Comment:
   replaced it in https://github.com/apache/nifi-minifi-cpp/pull/1383/commits/176864bc890da609911197adcc314b61d91e624b



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

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

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


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

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


##########
extensions/http-curl/processors/InvokeHTTP.h:
##########
@@ -127,52 +130,37 @@ class InvokeHTTP : public core::Processor {
 
   EXTENSIONAPI static constexpr const char* STATUS_CODE = "invokehttp.status.code";
   EXTENSIONAPI static constexpr const char* STATUS_MESSAGE = "invokehttp.status.message";
-  EXTENSIONAPI static constexpr const char* RESPONSE_BODY = "invokehttp.response.body";
   EXTENSIONAPI static constexpr const char* REQUEST_URL = "invokehttp.request.url";
   EXTENSIONAPI static constexpr const char* TRANSACTION_ID = "invokehttp.tx.id";
-  EXTENSIONAPI static constexpr const char* REMOTE_DN = "invokehttp.remote.dn";
-  EXTENSIONAPI static constexpr const char* EXCEPTION_CLASS = "invokehttp.java.exception.class";
-  EXTENSIONAPI static constexpr const char* EXCEPTION_MESSAGE = "invokehttp.java.exception.message";
 
-  void onTrigger(const std::shared_ptr<core::ProcessContext> &context, const std::shared_ptr<core::ProcessSession> &session) override;
+  void onTrigger(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSession>& session) override;
   void initialize() override;
-  void onSchedule(const std::shared_ptr<core::ProcessContext> &context, const std::shared_ptr<core::ProcessSessionFactory> &sessionFactory) override;
+  void onSchedule(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSessionFactory>& sessionFactory) override;
 
  private:
-  /**
-   * Routes the flowfile to the proper destination
-   * @param request request flow file record
-   * @param response response flow file record
-   * @param session process session
-   * @param context process context
-   * @param isSuccess success code or not
-   * @param statuscode http response code.
-   */
-  void route(const std::shared_ptr<core::FlowFile> &request, const std::shared_ptr<core::FlowFile> &response, const std::shared_ptr<core::ProcessSession> &session,
-             const std::shared_ptr<core::ProcessContext> &context, bool is_success, int64_t status_code);
-  bool shouldEmitFlowFile() const;
+  void route(const std::shared_ptr<core::FlowFile>& request, const std::shared_ptr<core::FlowFile>& response, const std::shared_ptr<core::ProcessSession>& session,
+             const std::shared_ptr<core::ProcessContext>& context, bool is_success, int64_t status_code);
+  static bool shouldEmitFlowFile(minifi::extensions::curl::HTTPClient& client);
+  void onTriggerWithClient(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSession>& session, minifi::extensions::curl::HTTPClient& client);
   [[nodiscard]] bool appendHeaders(const core::FlowFile& flow_file, /*std::invocable<std::string, std::string>*/ auto append_header);
 
-  std::shared_ptr<minifi::controllers::SSLContextService> ssl_context_service_;
-  std::string method_;
-  std::string url_;
-  bool date_header_include_{true};
+
+  void setupMembersFromProperties(const core::ProcessContext& context);
+  std::unique_ptr<minifi::extensions::curl::HTTPClient> createHTTPClientFromPropertiesAndMembers(const core::ProcessContext& context) const;
+
   std::optional<utils::Regex> attributes_to_send_;
-  std::chrono::milliseconds connect_timeout_ms_{20000};
-  std::chrono::milliseconds read_timeout_ms_{20000};
-  // attribute in which response body will be added
-  std::string put_attribute_name_;
+
+  std::optional<std::string> put_response_body_in_attribute_;
   bool always_output_response_{false};
-  std::string content_type_;
   bool use_chunked_encoding_{false};
   bool penalize_no_retry_{false};
-  // disabling peer verification makes susceptible for MITM attacks
-  bool disable_peer_verification_{false};
-  utils::HTTPProxy proxy_;
-  bool follow_redirects_{true};
   bool send_body_{true};
+
   InvalidHTTPHeaderFieldHandlingOption invalid_http_header_field_handling_strategy_;
+
   std::shared_ptr<core::logging::Logger> logger_{core::logging::LoggerFactory<InvokeHTTP>::getLogger()};
+  gsl::not_null<std::shared_ptr<utils::ResourceQueue<extensions::curl::HTTPClient>>> client_queue_ = gsl::make_not_null(
+      utils::ResourceQueue<extensions::curl::HTTPClient>::create(getMaxConcurrentTasks(), logger_));

Review Comment:
   I see -- this is only here to make `client_queue_` `not_null`.  I would prefer to leave it null at construction, and add a `gsl_Expects(client_queue_)` at the point where we use it, as constructing a dummy object with probably incorrect contents only to throw it away later feels wrong.  @szaszm may disagree, so just my 2¢.
   
   edit: same in `PutSplunkHTTP`



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

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

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


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

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


##########
extensions/http-curl/client/HTTPClient.cpp:
##########
@@ -32,62 +31,53 @@
 #include "range/v3/algorithm/all_of.hpp"
 #include "range/v3/action/transform.hpp"
 
-namespace org::apache::nifi::minifi::utils {
+using namespace std::literals::chrono_literals;
+
+namespace org::apache::nifi::minifi::extensions::curl {
 
-HTTPClient::HTTPClient(std::string url, const std::shared_ptr<minifi::controllers::SSLContextService> ssl_context_service)
+HTTPClient::HTTPClient(std::string url, std::shared_ptr<minifi::controllers::SSLContextService>  ssl_context_service)
     : core::Connectable("HTTPClient"),
-      ssl_context_service_(ssl_context_service),
+      ssl_context_service_(std::move(ssl_context_service)),
       url_(std::move(url)) {
-  http_session_ = curl_easy_init();
+  http_session_.reset(curl_easy_init());
 }
 
 HTTPClient::HTTPClient(const std::string& name, const utils::Identifier& uuid)
     : core::Connectable(name, uuid) {
-  http_session_ = curl_easy_init();
+  http_session_.reset(curl_easy_init());
 }
 
 HTTPClient::HTTPClient()
     : core::Connectable("HTTPClient") {
-  http_session_ = curl_easy_init();
+  http_session_.reset(curl_easy_init());
 }
 
-void HTTPClient::addFormPart(const std::string& content_type, const std::string& name, HTTPUploadCallback* read_callback, const std::optional<std::string>& filename) {
+void HTTPClient::addFormPart(const std::string& content_type, const std::string& name, std::unique_ptr<utils::HTTPUploadCallback>&& form_callback, const std::optional<std::string>& filename) {

Review Comment:
   makes sense, I fixed these in https://github.com/apache/nifi-minifi-cpp/pull/1383/commits/3edf5fcfd603463b67bd6c072fa0128bbb861787#diff-475912c726caf957fa46ce1b55d3e845e89aa19b4c7caf2cd4a0ae1f860cbdd5R98



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

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

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


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

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


##########
docker/test/integration/MiNiFi_integration_test_driver.py:
##########
@@ -132,7 +132,9 @@ def generate_input_port_for_remote_process_group(remote_process_group, name):
         input_port_node.set_uuid(uuid.uuid3(remote_process_group.get_uuid(), "input_port"))
         return input_port_node
 
-    def add_test_data(self, path, test_data, file_name=str(uuid.uuid4())):
+    def add_test_data(self, path, test_data, file_name=None):
+        if file_name is None:
+            file_name = str(uuid.uuid4())

Review Comment:
   Oh I see, thanks, good to know!



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

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

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


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

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


##########
extensions/script/ExecuteScript.h:
##########
@@ -185,23 +124,11 @@ class ExecuteScript : public core::Processor {
 
   ScriptEngineFactory engine_factory_;
 #ifdef LUA_SUPPORT
-  std::unique_ptr<ScriptEngineQueue<lua::LuaScriptEngine>> script_engine_q_;
+  std::shared_ptr<utils::ResourceQueue<lua::LuaScriptEngine>> lua_script_engine_queue_;

Review Comment:
   why did `lua_script_engine_queue_` change from a `unique_ptr` to a `shared_ptr`?



##########
libminifi/test/unit/ResourceQueueTests.cpp:
##########
@@ -0,0 +1,90 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include <chrono>
+#include <set>
+
+#include "ResourceQueue.h"
+#include "../TestBase.h"
+#include "../Catch.h"
+#include "logging/LoggerConfiguration.h"
+
+using namespace std::literals::chrono_literals;
+
+namespace org::apache::nifi::minifi::utils::testing {
+
+TEST_CASE("maximum_number_of_creatable_resources", "[utils::ResourceQueue]") {
+  std::shared_ptr<core::logging::Logger> logger_{core::logging::LoggerFactory<ResourceQueue<int>>::getLogger()};
+  LogTestController::getInstance().setTrace<ResourceQueue<int>>();
+
+  std::set<int> resources_created;
+
+  auto worker = [&](int value, const std::shared_ptr<ResourceQueue<int>>& resource_queue) {
+    auto resource = resource_queue->getResource([value]{return std::make_unique<int>(value);});
+    std::this_thread::sleep_for(10ms);
+    resources_created.emplace(*resource);
+  };
+
+  SECTION("Maximum 2 resources") {
+    auto resource_queue = ResourceQueue<int>::create(2, logger_);
+    std::thread thread_one{[&] { worker(1, resource_queue); }};
+    std::thread thread_two{[&] { worker(2, resource_queue); }};
+    std::thread thread_three{[&] { worker(3, resource_queue); }};
+
+    thread_one.join();
+    thread_two.join();
+    thread_three.join();
+
+    CHECK(!resources_created.empty());
+    CHECK(resources_created.size() <= 2);
+  }
+
+
+  SECTION("No Maximum resources") {
+    auto resource_queue = ResourceQueue<int>::create(2, logger_);

Review Comment:
   typo: 
   ```suggestion
       auto resource_queue = ResourceQueue<int>::create(std::nullopt, logger_);
   ```



##########
libminifi/test/unit/ResourceQueueTests.cpp:
##########
@@ -0,0 +1,90 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include <chrono>
+#include <set>
+
+#include "ResourceQueue.h"
+#include "../TestBase.h"
+#include "../Catch.h"
+#include "logging/LoggerConfiguration.h"
+
+using namespace std::literals::chrono_literals;
+
+namespace org::apache::nifi::minifi::utils::testing {
+
+TEST_CASE("maximum_number_of_creatable_resources", "[utils::ResourceQueue]") {
+  std::shared_ptr<core::logging::Logger> logger_{core::logging::LoggerFactory<ResourceQueue<int>>::getLogger()};
+  LogTestController::getInstance().setTrace<ResourceQueue<int>>();
+
+  std::set<int> resources_created;
+
+  auto worker = [&](int value, const std::shared_ptr<ResourceQueue<int>>& resource_queue) {
+    auto resource = resource_queue->getResource([value]{return std::make_unique<int>(value);});
+    std::this_thread::sleep_for(10ms);
+    resources_created.emplace(*resource);
+  };
+
+  SECTION("Maximum 2 resources") {
+    auto resource_queue = ResourceQueue<int>::create(2, logger_);
+    std::thread thread_one{[&] { worker(1, resource_queue); }};
+    std::thread thread_two{[&] { worker(2, resource_queue); }};
+    std::thread thread_three{[&] { worker(3, resource_queue); }};
+
+    thread_one.join();
+    thread_two.join();
+    thread_three.join();
+
+    CHECK(!resources_created.empty());
+    CHECK(resources_created.size() <= 2);
+  }
+
+
+  SECTION("No Maximum resources") {
+    auto resource_queue = ResourceQueue<int>::create(2, logger_);
+    std::thread thread_one{[&] { worker(1, resource_queue); }};
+    std::thread thread_two{[&] { worker(2, resource_queue); }};
+    std::thread thread_three{[&] { worker(3, resource_queue); }};
+
+    thread_one.join();
+    thread_two.join();
+    thread_three.join();
+
+    CHECK(!resources_created.empty());
+    CHECK(resources_created.size() <= 3);

Review Comment:
   This is a weak check, as it was passing in spite of the typo.  Maybe split the two sections into separate test cases, and save the resource wrapper instead of the resource in the unlimited case?  There is probably a better way.



##########
extensions/http-curl/tests/unit/ConnectionCountingServer.h:
##########
@@ -0,0 +1,149 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#pragma once
+
+#include <array>
+#include <vector>
+#include <string>
+#include <set>
+#include "CivetServer.h"
+
+namespace org::apache::nifi::minifi::extensions::curl::testing {
+
+namespace details {
+
+class NumberedMethodResponder : public CivetHandler {
+ public:
+  explicit NumberedMethodResponder(std::set<utils::SmallString<36>>& connections) : connections_(connections) {}
+
+  bool handleGet(CivetServer*, struct mg_connection* conn) override {
+    sendNumberedMessage("GET", conn);
+    return true;
+  }
+
+  bool handlePost(CivetServer*, struct mg_connection* conn) override {
+    sendNumberedMessage("POST", conn);
+    return true;
+  }
+
+  bool handlePut(CivetServer*, struct mg_connection* conn) override {
+    sendNumberedMessage("PUT", conn);
+    return true;
+  }
+
+  bool handleHead(CivetServer*, struct mg_connection* conn) override {
+    sendNumberedMessage("HEAD", conn);
+    return true;
+  }
+
+ private:
+  void sendNumberedMessage(std::string body, struct mg_connection* conn) {
+    saveConnectionId(conn);
+    body.append(std::to_string(response_id_));
+    mg_printf(conn, "HTTP/1.1 200 OK\r\n");
+    mg_printf(conn, "Content-length: %lu\r\n", body.length());
+    mg_printf(conn, "Response-number: %" PRIu64 "\r\n", response_id_);
+    mg_printf(conn, "\r\n");
+    mg_printf(conn, body.data(), body.length());
+    ++response_id_;
+  }
+
+  void saveConnectionId(struct mg_connection* conn) {
+    auto user_connection_data = reinterpret_cast<utils::SmallString<36>*>(mg_get_user_connection_data(conn));
+    connections_.emplace(*user_connection_data);

Review Comment:
   an assertion here that `user_connection_data` is not null would make me sleep better



##########
extensions/http-curl/tests/unit/ConnectionCountingServer.h:
##########
@@ -0,0 +1,149 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#pragma once
+
+#include <array>
+#include <vector>
+#include <string>
+#include <set>
+#include "CivetServer.h"
+
+namespace org::apache::nifi::minifi::extensions::curl::testing {
+
+namespace details {
+
+class NumberedMethodResponder : public CivetHandler {
+ public:
+  explicit NumberedMethodResponder(std::set<utils::SmallString<36>>& connections) : connections_(connections) {}
+
+  bool handleGet(CivetServer*, struct mg_connection* conn) override {
+    sendNumberedMessage("GET", conn);
+    return true;
+  }
+
+  bool handlePost(CivetServer*, struct mg_connection* conn) override {
+    sendNumberedMessage("POST", conn);
+    return true;
+  }
+
+  bool handlePut(CivetServer*, struct mg_connection* conn) override {
+    sendNumberedMessage("PUT", conn);
+    return true;
+  }
+
+  bool handleHead(CivetServer*, struct mg_connection* conn) override {
+    sendNumberedMessage("HEAD", conn);
+    return true;
+  }
+
+ private:
+  void sendNumberedMessage(std::string body, struct mg_connection* conn) {
+    saveConnectionId(conn);
+    body.append(std::to_string(response_id_));
+    mg_printf(conn, "HTTP/1.1 200 OK\r\n");
+    mg_printf(conn, "Content-length: %lu\r\n", body.length());
+    mg_printf(conn, "Response-number: %" PRIu64 "\r\n", response_id_);
+    mg_printf(conn, "\r\n");
+    mg_printf(conn, body.data(), body.length());
+    ++response_id_;
+  }
+
+  void saveConnectionId(struct mg_connection* conn) {
+    auto user_connection_data = reinterpret_cast<utils::SmallString<36>*>(mg_get_user_connection_data(conn));
+    connections_.emplace(*user_connection_data);
+  }
+
+  uint64_t response_id_ = 0;
+  std::set<utils::SmallString<36>>& connections_;
+};
+
+class ReverseBodyPostHandler : public CivetHandler {
+ public:
+  explicit ReverseBodyPostHandler(std::set<utils::SmallString<36>>& connections) : connections_(connections) {}
+
+  bool handlePost(CivetServer* /*server*/, struct mg_connection* conn) override {
+    saveConnectionId(conn);
+    std::vector<char> request_body;
+    request_body.reserve(2048);
+    size_t read_size = mg_read(conn, request_body.data(), 2048);

Review Comment:
   we could add an assertion that `read_size` is less than 2048, as that should be true in all tests



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

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

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


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

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


##########
extensions/http-curl/processors/InvokeHTTP.h:
##########
@@ -127,52 +130,37 @@ class InvokeHTTP : public core::Processor {
 
   EXTENSIONAPI static constexpr const char* STATUS_CODE = "invokehttp.status.code";
   EXTENSIONAPI static constexpr const char* STATUS_MESSAGE = "invokehttp.status.message";
-  EXTENSIONAPI static constexpr const char* RESPONSE_BODY = "invokehttp.response.body";
   EXTENSIONAPI static constexpr const char* REQUEST_URL = "invokehttp.request.url";
   EXTENSIONAPI static constexpr const char* TRANSACTION_ID = "invokehttp.tx.id";
-  EXTENSIONAPI static constexpr const char* REMOTE_DN = "invokehttp.remote.dn";
-  EXTENSIONAPI static constexpr const char* EXCEPTION_CLASS = "invokehttp.java.exception.class";
-  EXTENSIONAPI static constexpr const char* EXCEPTION_MESSAGE = "invokehttp.java.exception.message";
 
-  void onTrigger(const std::shared_ptr<core::ProcessContext> &context, const std::shared_ptr<core::ProcessSession> &session) override;
+  void onTrigger(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSession>& session) override;
   void initialize() override;
-  void onSchedule(const std::shared_ptr<core::ProcessContext> &context, const std::shared_ptr<core::ProcessSessionFactory> &sessionFactory) override;
+  void onSchedule(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSessionFactory>& sessionFactory) override;
 
  private:
-  /**
-   * Routes the flowfile to the proper destination
-   * @param request request flow file record
-   * @param response response flow file record
-   * @param session process session
-   * @param context process context
-   * @param isSuccess success code or not
-   * @param statuscode http response code.
-   */
-  void route(const std::shared_ptr<core::FlowFile> &request, const std::shared_ptr<core::FlowFile> &response, const std::shared_ptr<core::ProcessSession> &session,
-             const std::shared_ptr<core::ProcessContext> &context, bool is_success, int64_t status_code);
-  bool shouldEmitFlowFile() const;
+  void route(const std::shared_ptr<core::FlowFile>& request, const std::shared_ptr<core::FlowFile>& response, const std::shared_ptr<core::ProcessSession>& session,
+             const std::shared_ptr<core::ProcessContext>& context, bool is_success, int64_t status_code);
+  static bool shouldEmitFlowFile(minifi::extensions::curl::HTTPClient& client);
+  void onTriggerWithClient(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSession>& session, minifi::extensions::curl::HTTPClient& client);
   [[nodiscard]] bool appendHeaders(const core::FlowFile& flow_file, /*std::invocable<std::string, std::string>*/ auto append_header);
 
-  std::shared_ptr<minifi::controllers::SSLContextService> ssl_context_service_;
-  std::string method_;
-  std::string url_;
-  bool date_header_include_{true};
+
+  void setupMembersFromProperties(const core::ProcessContext& context);
+  std::unique_ptr<minifi::extensions::curl::HTTPClient> createHTTPClientFromPropertiesAndMembers(const core::ProcessContext& context) const;
+
   std::optional<utils::Regex> attributes_to_send_;
-  std::chrono::milliseconds connect_timeout_ms_{20000};
-  std::chrono::milliseconds read_timeout_ms_{20000};
-  // attribute in which response body will be added
-  std::string put_attribute_name_;
+
+  std::optional<std::string> put_response_body_in_attribute_;
   bool always_output_response_{false};
-  std::string content_type_;
   bool use_chunked_encoding_{false};
   bool penalize_no_retry_{false};
-  // disabling peer verification makes susceptible for MITM attacks
-  bool disable_peer_verification_{false};
-  utils::HTTPProxy proxy_;
-  bool follow_redirects_{true};
   bool send_body_{true};
+
   InvalidHTTPHeaderFieldHandlingOption invalid_http_header_field_handling_strategy_;
+
   std::shared_ptr<core::logging::Logger> logger_{core::logging::LoggerFactory<InvokeHTTP>::getLogger()};
+  gsl::not_null<std::shared_ptr<utils::ResourceQueue<extensions::curl::HTTPClient>>> client_queue_ = gsl::make_not_null(
+      utils::ResourceQueue<extensions::curl::HTTPClient>::create(getMaxConcurrentTasks(), logger_));

Review Comment:
   makes sense, it even reduces the boilerplate https://github.com/apache/nifi-minifi-cpp/pull/1383/commits/044ce269808a28e5a6eaeca34fb30747426a6ab7#diff-5880044259311bdd9e3a284e900759d3e34098641e21c89f78bdace17f4e63c7R163  https://github.com/apache/nifi-minifi-cpp/pull/1383/commits/044ce269808a28e5a6eaeca34fb30747426a6ab7#diff-66ae832b17ef9e3a1f26655cb827e35f9e663eb97f2923c913333989c160f9f6R76 



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

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

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


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

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


##########
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:
   makes it much cleaner :+1: , https://github.com/apache/nifi-minifi-cpp/pull/1383/commits/044ce269808a28e5a6eaeca34fb30747426a6ab7#diff-5b5811eda1c034612f091f1dfe2d4631d0c7df105969f9c12737224eae603a30R44



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

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

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


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

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


##########
libminifi/test/unit/ResourceQueueTests.cpp:
##########
@@ -0,0 +1,90 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include <chrono>
+#include <set>
+
+#include "ResourceQueue.h"
+#include "../TestBase.h"
+#include "../Catch.h"
+#include "logging/LoggerConfiguration.h"
+
+using namespace std::literals::chrono_literals;
+
+namespace org::apache::nifi::minifi::utils::testing {
+
+TEST_CASE("maximum_number_of_creatable_resources", "[utils::ResourceQueue]") {
+  std::shared_ptr<core::logging::Logger> logger_{core::logging::LoggerFactory<ResourceQueue<int>>::getLogger()};
+  LogTestController::getInstance().setTrace<ResourceQueue<int>>();
+
+  std::set<int> resources_created;
+
+  auto worker = [&](int value, const std::shared_ptr<ResourceQueue<int>>& resource_queue) {
+    auto resource = resource_queue->getResource([value]{return std::make_unique<int>(value);});
+    std::this_thread::sleep_for(10ms);
+    resources_created.emplace(*resource);
+  };
+
+  SECTION("Maximum 2 resources") {
+    auto resource_queue = ResourceQueue<int>::create(2, logger_);
+    std::thread thread_one{[&] { worker(1, resource_queue); }};
+    std::thread thread_two{[&] { worker(2, resource_queue); }};
+    std::thread thread_three{[&] { worker(3, resource_queue); }};
+
+    thread_one.join();
+    thread_two.join();
+    thread_three.join();
+
+    CHECK(!resources_created.empty());
+    CHECK(resources_created.size() <= 2);
+  }
+
+
+  SECTION("No Maximum resources") {
+    auto resource_queue = ResourceQueue<int>::create(2, logger_);
+    std::thread thread_one{[&] { worker(1, resource_queue); }};
+    std::thread thread_two{[&] { worker(2, resource_queue); }};
+    std::thread thread_three{[&] { worker(3, resource_queue); }};
+
+    thread_one.join();
+    thread_two.join();
+    thread_three.join();
+
+    CHECK(!resources_created.empty());
+    CHECK(resources_created.size() <= 3);

Review Comment:
   you are right :+1: I've done both of those in https://github.com/apache/nifi-minifi-cpp/pull/1383/commits/b85658fa591dea6f93cb8877f1c38c34039a0a19



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

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

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


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

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


##########
extensions/http-curl/client/RequestHeaders.h:
##########
@@ -0,0 +1,58 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#pragma once
+
+#include <unordered_map>
+#include <memory>
+#include <optional>
+#include <string>
+#include <string_view>
+
+#ifdef WIN32
+#pragma comment(lib, "wldap32.lib" )
+#pragma comment(lib, "crypt32.lib" )
+#pragma comment(lib, "Ws2_32.lib")
+
+#define CURL_STATICLIB
+#include <curl/curl.h>
+#else
+#include <curl/curl.h>
+#endif
+
+namespace org::apache::nifi::minifi::extensions::curl {
+class RequestHeaders {
+ public:
+  RequestHeaders() = default;
+
+  void appendHeader(std::string key, std::string value);
+
+  void disableExpectHeader();
+
+  [[nodiscard]] std::unique_ptr<struct curl_slist, decltype(&curl_slist_free_all)> get() const;
+  [[nodiscard]] auto size() const { return headers_.size(); }
+  [[nodiscard]] bool empty() const;
+
+  std::string& operator[](const std::string& key);
+  std::string& operator[](std::string&& key);
+
+  [[nodiscard]] bool contains(const std::string& key) const;
+  void erase(const std::string& key);
+
+ private:
+  std::unordered_map<std::string, std::string> headers_;
+};

Review Comment:
   I think it would be simpler, so yes, if you don't mind.



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

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

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


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

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


##########
libminifi/test/unit/ResourceQueueTests.cpp:
##########
@@ -0,0 +1,90 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include <chrono>
+#include <set>
+
+#include "ResourceQueue.h"
+#include "../TestBase.h"
+#include "../Catch.h"
+#include "logging/LoggerConfiguration.h"
+
+using namespace std::literals::chrono_literals;
+
+namespace org::apache::nifi::minifi::utils::testing {
+
+TEST_CASE("maximum_number_of_creatable_resources", "[utils::ResourceQueue]") {
+  std::shared_ptr<core::logging::Logger> logger_{core::logging::LoggerFactory<ResourceQueue<int>>::getLogger()};
+  LogTestController::getInstance().setTrace<ResourceQueue<int>>();
+
+  std::set<int> resources_created;
+
+  auto worker = [&](int value, const std::shared_ptr<ResourceQueue<int>>& resource_queue) {
+    auto resource = resource_queue->getResource([value]{return std::make_unique<int>(value);});
+    std::this_thread::sleep_for(10ms);
+    resources_created.emplace(*resource);
+  };
+
+  SECTION("Maximum 2 resources") {
+    auto resource_queue = ResourceQueue<int>::create(2, logger_);
+    std::thread thread_one{[&] { worker(1, resource_queue); }};
+    std::thread thread_two{[&] { worker(2, resource_queue); }};
+    std::thread thread_three{[&] { worker(3, resource_queue); }};
+
+    thread_one.join();
+    thread_two.join();
+    thread_three.join();
+
+    CHECK(!resources_created.empty());
+    CHECK(resources_created.size() <= 2);
+  }
+
+
+  SECTION("No Maximum resources") {
+    auto resource_queue = ResourceQueue<int>::create(2, logger_);
+    std::thread thread_one{[&] { worker(1, resource_queue); }};
+    std::thread thread_two{[&] { worker(2, resource_queue); }};
+    std::thread thread_three{[&] { worker(3, resource_queue); }};
+
+    thread_one.join();
+    thread_two.join();
+    thread_three.join();
+
+    CHECK(!resources_created.empty());
+    CHECK(resources_created.size() <= 3);

Review Comment:
   you need to reset the logger, as the other section prints "Waiting for resource"
   
   or, probably better, you could look for "Number of instances: 3" (but resetting the logger is a good idea in this case, too)



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

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

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


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

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


##########
extensions/http-curl/client/HTTPClient.h:
##########
@@ -51,17 +53,28 @@
 #include "core/logging/Logger.h"
 #include "core/logging/LoggerConfiguration.h"
 
-namespace org::apache::nifi::minifi::utils {
+namespace org::apache::nifi::minifi::extensions::curl {
 
-/**
- * Purpose and Justification: Pull the basics for an HTTPClient into a self contained class. Simply provide
- * the URL and an SSLContextService ( can be null).
- *
- * Since several portions of the code have been relying on curl, we can encapsulate most CURL HTTP
- * operations here without maintaining it everywhere. Further, this will help with testing as we
- * only need to to test our usage of CURL once
- */
-class HTTPClient : public BaseHTTPClient, public core::Connectable {
+struct KeepAliveProbeData {
+  std::chrono::seconds keep_alive_delay;
+  std::chrono::seconds keep_alive_interval;
+};
+
+struct HTTPResponseData {
+  std::vector<char> response_body;
+  utils::HTTPHeaderResponse header_response;
+  char* response_content_type;
+  int64_t response_code;

Review Comment:
   not at all, fixed in https://github.com/apache/nifi-minifi-cpp/pull/1383/commits/3edf5fcfd603463b67bd6c072fa0128bbb861787#diff-475912c726caf957fa46ce1b55d3e845e89aa19b4c7caf2cd4a0ae1f860cbdd5R56-R57



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

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

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


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

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


##########
extensions/http-curl/client/HTTPClient.h:
##########
@@ -270,48 +249,33 @@ class HTTPClient : public BaseHTTPClient, public core::Connectable {
 
   void configure_secure_connection(CURL *http_session);
 
-  bool isSecure(const std::string &url);
+  std::chrono::milliseconds getAbsoluteTimeout() const { return 3*read_timeout_; }
 
-  HTTPReadCallback content_;
+  utils::HTTPReadCallback content_{std::numeric_limits<size_t>::max()};
 
   std::shared_ptr<minifi::controllers::SSLContextService> ssl_context_service_;
   std::string url_;
-  std::chrono::milliseconds connect_timeout_ms_{30000};
-  // read timeout.
-  std::chrono::milliseconds read_timeout_ms_{30000};
-  char *content_type_str_{nullptr};
-  std::string content_type_;
-  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{CURLE_OK};
-
-  CURL* http_session_{nullptr};
-  curl_mime* form_{nullptr};
-
   std::string method_;
 
-  std::chrono::milliseconds keep_alive_probe_{-1};
+  std::chrono::milliseconds connect_timeout_{std::chrono::seconds(30)};
+  std::chrono::milliseconds read_timeout_{std::chrono::seconds(30)};
 
-  std::chrono::milliseconds keep_alive_idle_{-1};
+  HTTPResponseData response_data_;
 
-  struct BasicAuthCredentials {
-    BasicAuthCredentials(std::string username, std::string password) : username(std::move(username)), password(std::move(password)) {}
+  CURLcode res_{CURLE_OK};
 
-    std::string username;
-    std::string password;
-  };
+  RequestHeaders request_headers_;
 
-  std::optional<BasicAuthCredentials> username_password_;
+  std::unique_ptr<CURL, decltype(&curl_easy_cleanup)> http_session_{nullptr, curl_easy_cleanup};
+  std::unique_ptr<curl_mime, decltype(&curl_mime_free)> form_{nullptr, curl_mime_free};

Review Comment:
   good idea, I've replaced them in https://github.com/apache/nifi-minifi-cpp/commit/3edf5fcfd603463b67bd6c072fa0128bbb861787#diff-475912c726caf957fa46ce1b55d3e845e89aa19b4c7caf2cd4a0ae1f860cbdd5R241-R242
   Didnt remove the include from the header because there is a member that still depends on it `CURLcode res_{CURLE_OK}`



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

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

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


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

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


##########
extensions/http-curl/processors/InvokeHTTP.cpp:
##########
@@ -46,31 +45,31 @@ namespace org::apache::nifi::minifi::processors {
 std::string InvokeHTTP::DefaultContentType = "application/octet-stream";
 
 const core::Property InvokeHTTP::Method("HTTP Method", "HTTP request method (GET, POST, PUT, PATCH, DELETE, HEAD, OPTIONS). "
-                                  "Arbitrary methods are also supported. Methods other than POST, PUT and PATCH will be sent without a message body.",
-                                  "GET");
+                                                       "Arbitrary methods are also supported. Methods other than POST, PUT and PATCH will be sent without a message body.",
+                                        "GET");

Review Comment:
   fixed the indentation issues in https://github.com/apache/nifi-minifi-cpp/commit/3edf5fcfd603463b67bd6c072fa0128bbb861787



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

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

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


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

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


##########
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:
   fixed in https://github.com/apache/nifi-minifi-cpp/pull/1383/commits/044ce269808a28e5a6eaeca34fb30747426a6ab7#diff-e83ea756e38995b3194ecc99d0a64fc19c8ef992fb7a575a34def27a67f7819aR311



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

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

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


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

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


##########
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:
   you are right, removed these in https://github.com/apache/nifi-minifi-cpp/pull/1383/commits/044ce269808a28e5a6eaeca34fb30747426a6ab7#diff-4e08bc45485eac46e59652e031a79a51dca96dc3a08fa5bf6ee8d2b5603bda6eL68



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

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

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


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

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


##########
extensions/http-curl/processors/InvokeHTTP.cpp:
##########
@@ -83,165 +84,188 @@ const core::Property InvokeHTTP::ProxyUsername(
     core::PropertyBuilder::createProperty("invokehttp-proxy-username", "Proxy Username")->withDescription("Username to set when authenticating against proxy")->isRequired(false)->build());
 const core::Property InvokeHTTP::ProxyPassword(
     core::PropertyBuilder::createProperty("invokehttp-proxy-password", "Proxy Password")->withDescription("Password to set when authenticating against proxy")->isRequired(false)->build());
-const core::Property InvokeHTTP::ContentType("Content-type", "The Content-Type to specify for when content is being transmitted through a PUT, "
-                                       "POST or PATCH. In the case of an empty value after evaluating an expression language expression, "
-                                       "Content-Type defaults to",
-                                       "application/octet-stream");
+const core::Property InvokeHTTP::ContentType("Content-type",
+    "The Content-Type to specify for when content is being transmitted through a PUT, "
+    "POST or PATCH. In the case of an empty value after evaluating an expression language expression, "
+    "Content-Type defaults to",
+    "application/octet-stream");
 const core::Property InvokeHTTP::SendBody(
     core::PropertyBuilder::createProperty("send-message-body", "Send Body")
-      ->withDescription("DEPRECATED. Only kept for backwards compatibility, no functionality is included.")
-      ->withDefaultValue<bool>(true)
-      ->build());
+        ->withDescription("DEPRECATED. Only kept for backwards compatibility, no functionality is included.")
+        ->withDefaultValue<bool>(true)
+        ->build());
 const core::Property InvokeHTTP::SendMessageBody(
     core::PropertyBuilder::createProperty("Send Message Body")
-      ->withDescription("If true, sends the HTTP message body on POST/PUT/PATCH requests (default). "
-                        "If false, suppresses the message body and content-type header for these requests.")
-      ->withDefaultValue<bool>(true)
-      ->build());
-const core::Property InvokeHTTP::UseChunkedEncoding("Use Chunked Encoding", "When POST'ing, PUT'ing or PATCH'ing content set this property to true in order to not pass the 'Content-length' header"
-                                              " and instead send 'Transfer-Encoding' with a value of 'chunked'. This will enable the data transfer mechanism which was introduced in HTTP 1.1 "
-                                              "to pass data of unknown lengths in chunks.",
-                                              "false");
-const core::Property InvokeHTTP::PropPutOutputAttributes("Put Response Body in Attribute", "If set, the response body received back will be put into an attribute of the original "
-                                                   "FlowFile instead of a separate FlowFile. The attribute key to put to is determined by evaluating value of this property. ",
-                                                   "");
-const core::Property InvokeHTTP::AlwaysOutputResponse("Always Output Response", "Will force a response FlowFile to be generated and routed to the 'Response' relationship "
-                                                "regardless of what the server status code received is ",
-                                                "false");
-const core::Property InvokeHTTP::PenalizeOnNoRetry("Penalize on \"No Retry\"", "Enabling this property will penalize FlowFiles that are routed to the \"No Retry\" relationship.", "false");
+        ->withDescription("If true, sends the HTTP message body on POST/PUT/PATCH requests (default). "
+                          "If false, suppresses the message body and content-type header for these requests.")
+        ->withDefaultValue<bool>(true)
+        ->build());
+const core::Property InvokeHTTP::UseChunkedEncoding("Use Chunked Encoding",
+    "When POST'ing, PUT'ing or PATCH'ing content set this property to true in order to not pass the 'Content-length' header"
+    " and instead send 'Transfer-Encoding' with a value of 'chunked'."
+    " This will enable the data transfer mechanism which was introduced in HTTP 1.1 to pass data of unknown lengths in chunks.",
+    "false");
+const core::Property InvokeHTTP::PutResponseBodyInAttribute("Put Response Body in Attribute",
+    "If set, the response body received back will be put into an attribute of the original "
+    "FlowFile instead of a separate FlowFile. "
+    "The attribute key to put to is determined by evaluating value of this property. ",
+    "");
+const core::Property InvokeHTTP::AlwaysOutputResponse("Always Output Response",
+    "Will force a response FlowFile to be generated and routed to the 'Response' relationship regardless of what the server status code received is ",
+    "false");
+const core::Property InvokeHTTP::PenalizeOnNoRetry("Penalize on \"No Retry\"",
+    "Enabling this property will penalize FlowFiles that are routed to the \"No Retry\" relationship.",
+    "false");
 
 const core::Property InvokeHTTP::DisablePeerVerification("Disable Peer Verification", "Disables peer verification for the SSL session", "false");
 
 const core::Property InvokeHTTP::InvalidHTTPHeaderFieldHandlingStrategy(
     core::PropertyBuilder::createProperty("Invalid HTTP Header Field Handling Strategy")
-      ->withDescription("Indicates what should happen when an attribute's name is not a valid HTTP header field name. "
-        "Options: transform - invalid characters are replaced, fail - flow file is transferred to failure, drop - drops invalid attributes from HTTP message")
-      ->isRequired(true)
-      ->withDefaultValue<std::string>(toString(InvalidHTTPHeaderFieldHandlingOption::TRANSFORM))
-      ->withAllowableValues<std::string>(InvalidHTTPHeaderFieldHandlingOption::values())
-      ->build());
+        ->withDescription("Indicates what should happen when an attribute's name is not a valid HTTP header field name. "
+                          "Options: transform - invalid characters are replaced, fail - flow file is transferred to failure, drop - drops invalid attributes from HTTP message")
+        ->isRequired(true)
+        ->withDefaultValue<std::string>(toString(InvalidHTTPHeaderFieldHandlingOption::TRANSFORM))
+        ->withAllowableValues<std::string>(InvalidHTTPHeaderFieldHandlingOption::values())
+        ->build());
 
 
-const core::Relationship InvokeHTTP::Success("success", "The original FlowFile will be routed upon success (2xx status codes). "
-                                       "It will have new attributes detailing the success of the request.");
+const core::Relationship InvokeHTTP::Success("success",
+    "The original FlowFile will be routed upon success (2xx status codes). It will have new attributes detailing the success of the request.");
 
-const core::Relationship InvokeHTTP::RelResponse("response", "A Response FlowFile will be routed upon success (2xx status codes). "
-                                           "If the 'Always Output Response' property is true then the response will be sent "
-                                           "to this relationship regardless of the status code received.");
+const core::Relationship InvokeHTTP::RelResponse("response",
+    "A Response FlowFile will be routed upon success (2xx status codes). "
+    "If the 'Always Output Response' property is true then the response will be sent "
+    "to this relationship regardless of the status code received.");
 
-const core::Relationship InvokeHTTP::RelRetry("retry", "The original FlowFile will be routed on any status code that can be retried "
-                                        "(5xx status codes). It will have new attributes detailing the request.");
+const core::Relationship InvokeHTTP::RelRetry("retry",
+    "The original FlowFile will be routed on any status code that can be retried "
+    "(5xx status codes). It will have new attributes detailing the request.");
 
-const core::Relationship InvokeHTTP::RelNoRetry("no retry", "The original FlowFile will be routed on any status code that should NOT "
-                                          "be retried (1xx, 3xx, 4xx status codes). It will have new attributes detailing the request.");
+const core::Relationship InvokeHTTP::RelNoRetry("no retry",
+    "The original FlowFile will be routed on any status code that should NOT "
+    "be retried (1xx, 3xx, 4xx status codes). It will have new attributes detailing the request.");
 
-const core::Relationship InvokeHTTP::RelFailure("failure", "The original FlowFile will be routed on any type of connection failure, "
-                                          "timeout or general exception. It will have new attributes detailing the request.");
+const core::Relationship InvokeHTTP::RelFailure("failure",
+    "The original FlowFile will be routed on any type of connection failure, "
+    "timeout or general exception. It will have new attributes detailing the request.");
 
 void InvokeHTTP::initialize() {
   logger_->log_trace("Initializing InvokeHTTP");
   setSupportedProperties(properties());
   setSupportedRelationships(relationships());
 }
 
-void InvokeHTTP::onSchedule(const std::shared_ptr<core::ProcessContext> &context, const std::shared_ptr<core::ProcessSessionFactory>& /*sessionFactory*/) {
-  if (!context->getProperty(Method.getName(), method_)) {
-    logger_->log_debug("%s attribute is missing, so default value of %s will be used", Method.getName(), Method.getValue());
-    return;
-  }
+namespace {
+void setupClientTimeouts(extensions::curl::HTTPClient& client, const core::ProcessContext& context) {
+  if (auto connection_timeout = context.getProperty<core::TimePeriodValue>(InvokeHTTP::ConnectTimeout))
+    client.setConnectionTimeout(connection_timeout->getMilliseconds());
 
-  if (!context->getProperty(URL.getName(), url_)) {
-    logger_->log_debug("%s attribute is missing, so default value of %s will be used", URL.getName(), URL.getValue());
-    return;
-  }
+  if (auto read_timeout = context.getProperty<core::TimePeriodValue>(InvokeHTTP::ReadTimeout))
+    client.setReadTimeout(read_timeout->getMilliseconds());
+}
 
-  if (auto connect_timeout = context->getProperty<core::TimePeriodValue>(ConnectTimeout)) {
-    connect_timeout_ms_ =  connect_timeout->getMilliseconds();
-  } else {
-    logger_->log_debug("%s attribute is missing, so default value of %s will be used", ConnectTimeout.getName(), ConnectTimeout.getValue());
-    return;
+void setupClientProxy(extensions::curl::HTTPClient& client, const core::ProcessContext& context) {
+  utils::HTTPProxy proxy = {};
+  context.getProperty(InvokeHTTP::ProxyHost.getName(), proxy.host);
+  std::string port_str;
+  if (context.getProperty(InvokeHTTP::ProxyPort.getName(), port_str) && !port_str.empty()) {
+    core::Property::StringToInt(port_str, proxy.port);
   }
+  context.getProperty(InvokeHTTP::ProxyUsername.getName(), proxy.username);
+  context.getProperty(InvokeHTTP::ProxyPassword.getName(), proxy.password);
 
-  std::string content_type_str;
-  if (context->getProperty(ContentType.getName(), content_type_str)) {
-    content_type_ = content_type_str;
-  }
+  client.setHTTPProxy(proxy);
+}
 
-  if (auto read_timeout = context->getProperty<core::TimePeriodValue>(ReadTimeout)) {
-    read_timeout_ms_ =  read_timeout->getMilliseconds();
-  } else {
-    logger_->log_debug("%s attribute is missing, so default value of %s will be used", ReadTimeout.getName(), ReadTimeout.getValue());
-  }
+void setupClientPeerVerification(extensions::curl::HTTPClient& client, const core::ProcessContext& context) {
+  if (auto disable_peer_verification = context.getProperty<bool>(InvokeHTTP::DisablePeerVerification))
+    client.setPeerVerification(*disable_peer_verification);
+}
+
+void setupClientFollowRedirects(extensions::curl::HTTPClient& client, const core::ProcessContext& context) {
+  if (auto follow_redirects = context.getProperty<bool>(InvokeHTTP::FollowRedirects))
+    client.setFollowRedirects(*follow_redirects);
+}
 
-  std::string date_header_str;
-  if (!context->getProperty(DateHeader.getName(), date_header_str)) {
-    logger_->log_debug("%s attribute is missing, so default value of %s will be used", DateHeader.getName(), DateHeader.getValue());
+void setupClientContentType(extensions::curl::HTTPClient& client, const core::ProcessContext& context, bool send_body) {
+  if (auto content_type = context.getProperty(InvokeHTTP::ContentType)) {
+    if (send_body)
+      client.setContentType(*content_type);
   }
+}
 
-  date_header_include_ = utils::StringUtils::toBool(date_header_str).value_or(DateHeader.getValue());
+void setupClientTransferEncoding(extensions::curl::HTTPClient& client, bool use_chunked_encoding) {
+  if (use_chunked_encoding)
+    client.setRequestHeader("Transfer-Encoding", "chunked");

Review Comment:
   wow good catch, 
   the UseChunkedEncoding seems like a new bug, it disappeared somewhere during the refactor. fixed in https://github.com/apache/nifi-minifi-cpp/pull/1383/commits/044ce269808a28e5a6eaeca34fb30747426a6ab7#diff-5315e220af4eeb42d2fe04a9dee9eaffd26fef7ff21ffab614140d3351e7678eR225-R226
   
   The DateHeader was never used but fixed that aswell, https://github.com/apache/nifi-minifi-cpp/pull/1383/commits/044ce269808a28e5a6eaeca34fb30747426a6ab7#diff-99faae2cd43e40a705a685c1fb46ddffaace1246c280d3d1d4324e39fa574d4cR395-R424



##########
extensions/http-curl/tests/unit/InvokeHTTPTests.cpp:
##########
@@ -433,4 +414,76 @@ TEST_CASE("InvokeHTTP Attributes to Send uses full string matching, not substrin
   REQUIRE(LogTestController::getInstance().contains("key:header value:value2"));
   REQUIRE_FALSE(LogTestController::getInstance().contains("key:invalid", 0s));
 }
+
+TEST_CASE("HTTPTestsResponseBodyinAttribute", "[InvokeHTTP]") {
+  using minifi::processors::InvokeHTTP;
+
+  auto invoke_http = std::make_shared<InvokeHTTP>("InvokeHTTP");
+  test::SingleProcessorTestController test_controller{invoke_http};
+
+  minifi::extensions::curl::testing::ConnectionCountingServer connection_counting_server;
+
+  invoke_http->setProperty(InvokeHTTP::Method, "POST");
+  invoke_http->setProperty(InvokeHTTP::URL, "http://localhost:" + connection_counting_server.getPort()  + "/reverse");
+  invoke_http->setProperty(InvokeHTTP::PutResponseBodyInAttribute, "http.body");
+  const auto result = test_controller.trigger("data", {{"header1", "value1"}, {"header", "value2"}});
+  auto success_flow_files = result.at(InvokeHTTP::Success);
+  CHECK(result.at(InvokeHTTP::RelFailure).empty());
+  CHECK(result.at(InvokeHTTP::RelResponse).empty());
+  CHECK(result.at(InvokeHTTP::RelNoRetry).empty());
+  CHECK(result.at(InvokeHTTP::RelRetry).empty());
+  REQUIRE(success_flow_files.size() == 1);
+  CHECK(test_controller.plan->getContent(success_flow_files[0]) == "data");
+
+  auto http_type_attribute = success_flow_files[0]->getAttribute("http.body");
+  REQUIRE(http_type_attribute);
+  CHECK(*http_type_attribute == "atad");
+}
+
+TEST_CASE("HTTPTestsResponseBody", "[InvokeHTTP]") {
+  using minifi::processors::InvokeHTTP;
+
+  auto invoke_http = std::make_shared<InvokeHTTP>("InvokeHTTP");
+  test::SingleProcessorTestController test_controller{invoke_http};
+
+  minifi::extensions::curl::testing::ConnectionCountingServer connection_counting_server;
+
+  invoke_http->setProperty(InvokeHTTP::Method, "POST");
+  invoke_http->setProperty(InvokeHTTP::URL, "http://localhost:" + connection_counting_server.getPort()  + "/reverse");
+  invoke_http->setProperty(InvokeHTTP::SendBody, "true");

Review Comment:
   :+1: fixed in https://github.com/apache/nifi-minifi-cpp/pull/1383/commits/044ce269808a28e5a6eaeca34fb30747426a6ab7#diff-99faae2cd43e40a705a685c1fb46ddffaace1246c280d3d1d4324e39fa574d4cR483



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

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

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


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

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


##########
extensions/http-curl/tests/unit/ConnectionCountingServer.h:
##########
@@ -0,0 +1,149 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#pragma once
+
+#include <array>
+#include <vector>
+#include <string>
+#include <set>
+#include "CivetServer.h"
+
+namespace org::apache::nifi::minifi::extensions::curl::testing {
+
+namespace details {
+
+class NumberedMethodResponder : public CivetHandler {
+ public:
+  explicit NumberedMethodResponder(std::set<utils::SmallString<36>>& connections) : connections_(connections) {}
+
+  bool handleGet(CivetServer*, struct mg_connection* conn) override {
+    sendNumberedMessage("GET", conn);
+    return true;
+  }
+
+  bool handlePost(CivetServer*, struct mg_connection* conn) override {
+    sendNumberedMessage("POST", conn);
+    return true;
+  }
+
+  bool handlePut(CivetServer*, struct mg_connection* conn) override {
+    sendNumberedMessage("PUT", conn);
+    return true;
+  }
+
+  bool handleHead(CivetServer*, struct mg_connection* conn) override {
+    sendNumberedMessage("HEAD", conn);
+    return true;
+  }
+
+ private:
+  void sendNumberedMessage(std::string body, struct mg_connection* conn) {
+    saveConnectionId(conn);
+    body.append(std::to_string(response_id_));
+    mg_printf(conn, "HTTP/1.1 200 OK\r\n");
+    mg_printf(conn, "Content-length: %lu\r\n", body.length());
+    mg_printf(conn, "Response-number: %" PRIu64 "\r\n", response_id_);
+    mg_printf(conn, "\r\n");
+    mg_printf(conn, body.data(), body.length());
+    ++response_id_;
+  }
+
+  void saveConnectionId(struct mg_connection* conn) {
+    auto user_connection_data = reinterpret_cast<utils::SmallString<36>*>(mg_get_user_connection_data(conn));
+    connections_.emplace(*user_connection_data);
+  }
+
+  uint64_t response_id_ = 0;
+  std::set<utils::SmallString<36>>& connections_;
+};
+
+class ReverseBodyPostHandler : public CivetHandler {
+ public:
+  explicit ReverseBodyPostHandler(std::set<utils::SmallString<36>>& connections) : connections_(connections) {}
+
+  bool handlePost(CivetServer* /*server*/, struct mg_connection* conn) override {
+    saveConnectionId(conn);
+    std::vector<char> request_body;
+    request_body.reserve(2048);
+    size_t read_size = mg_read(conn, request_body.data(), 2048);

Review Comment:
   added in https://github.com/apache/nifi-minifi-cpp/pull/1383/commits/3f4aeec40af42fb9df1b15fbf0c2298664e77675#diff-49cbe304b911422ac56db05d81e536df1526f08edbe8a2d53aee8c34fa6fd3c3R86



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

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

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


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

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


##########
extensions/http-curl/tests/unit/ConnectionCountingServer.h:
##########
@@ -0,0 +1,149 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#pragma once
+
+#include <array>
+#include <vector>
+#include <string>
+#include <set>
+#include "CivetServer.h"
+
+namespace org::apache::nifi::minifi::extensions::curl::testing {
+
+namespace details {
+
+class NumberedMethodResponder : public CivetHandler {
+ public:
+  explicit NumberedMethodResponder(std::set<utils::SmallString<36>>& connections) : connections_(connections) {}
+
+  bool handleGet(CivetServer*, struct mg_connection* conn) override {
+    sendNumberedMessage("GET", conn);
+    return true;
+  }
+
+  bool handlePost(CivetServer*, struct mg_connection* conn) override {
+    sendNumberedMessage("POST", conn);
+    return true;
+  }
+
+  bool handlePut(CivetServer*, struct mg_connection* conn) override {
+    sendNumberedMessage("PUT", conn);
+    return true;
+  }
+
+  bool handleHead(CivetServer*, struct mg_connection* conn) override {
+    sendNumberedMessage("HEAD", conn);
+    return true;
+  }
+
+ private:
+  void sendNumberedMessage(std::string body, struct mg_connection* conn) {
+    saveConnectionId(conn);
+    body.append(std::to_string(response_id_));
+    mg_printf(conn, "HTTP/1.1 200 OK\r\n");
+    mg_printf(conn, "Content-length: %lu\r\n", body.length());
+    mg_printf(conn, "Response-number: %" PRIu64 "\r\n", response_id_);
+    mg_printf(conn, "\r\n");
+    mg_printf(conn, body.data(), body.length());
+    ++response_id_;
+  }
+
+  void saveConnectionId(struct mg_connection* conn) {
+    auto user_connection_data = reinterpret_cast<utils::SmallString<36>*>(mg_get_user_connection_data(conn));
+    connections_.emplace(*user_connection_data);

Review Comment:
   :sleeping_bed: :sleeping: https://github.com/apache/nifi-minifi-cpp/pull/1383/commits/3f4aeec40af42fb9df1b15fbf0c2298664e77675#diff-49cbe304b911422ac56db05d81e536df1526f08edbe8a2d53aee8c34fa6fd3c3R69



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