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

2022-08-26 Thread GitBox


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 
+#include 
+#include 
+#include 
+#include 
+#include 
+#include 
+#include 
+
+#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 ResourceQueue : public 
std::enable_shared_from_this> {
+ public:
+  class ResourceWrapper {
+   public:
+ResourceWrapper(std::weak_ptr> queue, 
std::unique_ptr&& 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> queue_;
+std::unique_ptr resource_;
+  };
+
+  static auto create(std::optional 
maximum_number_of_creatable_resources, std::shared_ptr 
logger);
+
+  template
+  [[nodiscard]] 
std::enable_if_t()>, 
ResourceWrapper> getResource(const Fn& create_resource) {
+std::unique_ptr 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 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 maximum_number_of_creatable_resources, 
std::shared_ptr logger)
+  : 
maximum_number_of_creatable_resources_(maximum_number_of_creatable_resources),
+logger_(std::move(logger)) {
+  }
+
+ private:
+  void returnResource(std::unique_ptr&& 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-unsubscr...@nifi.apache.org

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



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

2022-08-25 Thread GitBox


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 
+#include 
+#include 
+#include 
+#include "CivetServer.h"
+
+namespace org::apache::nifi::minifi::extensions::curl::testing {
+
+namespace details {
+
+class NumberedMethodResponder : public CivetHandler {
+ public:
+  explicit NumberedMethodResponder(std::set>& 
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*>(mg_get_user_connection_data(conn));
+connections_.emplace(*user_connection_data);
+  }
+
+  uint64_t response_id_ = 0;
+  std::set>& connections_;
+};
+
+class ReverseBodyPostHandler : public CivetHandler {
+ public:
+  explicit ReverseBodyPostHandler(std::set>& 
connections) : connections_(connections) {}
+
+  bool handlePost(CivetServer* /*server*/, struct mg_connection* conn) 
override {
+saveConnectionId(conn);
+std::vector 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-unsubscr...@nifi.apache.org

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



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

2022-08-25 Thread GitBox


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 
+#include 
+
+#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 
logger_{core::logging::LoggerFactory>::getLogger()};
+  LogTestController::getInstance().setTrace>();
+
+  std::set resources_created;
+
+  auto worker = [&](int value, const std::shared_ptr>& 
resource_queue) {
+auto resource = resource_queue->getResource([value]{return 
std::make_unique(value);});
+std::this_thread::sleep_for(10ms);
+resources_created.emplace(*resource);
+  };
+
+  SECTION("Maximum 2 resources") {
+auto resource_queue = ResourceQueue::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::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-unsubscr...@nifi.apache.org

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



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

2022-08-25 Thread GitBox


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 
+#include 
+
+#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 
logger_{core::logging::LoggerFactory>::getLogger()};
+  LogTestController::getInstance().setTrace>();
+
+  std::set resources_created;
+
+  auto worker = [&](int value, const std::shared_ptr>& 
resource_queue) {
+auto resource = resource_queue->getResource([value]{return 
std::make_unique(value);});
+std::this_thread::sleep_for(10ms);
+resources_created.emplace(*resource);
+  };
+
+  SECTION("Maximum 2 resources") {
+auto resource_queue = ResourceQueue::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::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-unsubscr...@nifi.apache.org

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



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

2022-08-25 Thread GitBox


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 
+#include 
+
+#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 
logger_{core::logging::LoggerFactory>::getLogger()};
+  LogTestController::getInstance().setTrace>();
+
+  std::set resources_created;
+
+  auto worker = [&](int value, const std::shared_ptr>& 
resource_queue) {
+auto resource = resource_queue->getResource([value]{return 
std::make_unique(value);});
+std::this_thread::sleep_for(10ms);
+resources_created.emplace(*resource);
+  };
+
+  SECTION("Maximum 2 resources") {
+auto resource_queue = ResourceQueue::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::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-unsubscr...@nifi.apache.org

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



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

2022-08-25 Thread GitBox


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 
+#include 
+#include 
+#include 
+#include "CivetServer.h"
+
+namespace org::apache::nifi::minifi::extensions::curl::testing {
+
+namespace details {
+
+class NumberedMethodResponder : public CivetHandler {
+ public:
+  explicit NumberedMethodResponder(std::set>& 
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*>(mg_get_user_connection_data(conn));
+connections_.emplace(*user_connection_data);
+  }
+
+  uint64_t response_id_ = 0;
+  std::set>& connections_;
+};
+
+class ReverseBodyPostHandler : public CivetHandler {
+ public:
+  explicit ReverseBodyPostHandler(std::set>& 
connections) : connections_(connections) {}
+
+  bool handlePost(CivetServer* /*server*/, struct mg_connection* conn) 
override {
+saveConnectionId(conn);
+std::vector 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-unsubscr...@nifi.apache.org

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



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

2022-08-25 Thread GitBox


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 
+#include 
+#include 
+#include 
+#include "CivetServer.h"
+
+namespace org::apache::nifi::minifi::extensions::curl::testing {
+
+namespace details {
+
+class NumberedMethodResponder : public CivetHandler {
+ public:
+  explicit NumberedMethodResponder(std::set>& 
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*>(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-unsubscr...@nifi.apache.org

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



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

2022-08-25 Thread GitBox


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 , const 
std::shared_ptr ) override;
+  void onTrigger(const std::shared_ptr& context, const 
std::shared_ptr& session) override;
   void initialize() override;
-  void onSchedule(const std::shared_ptr , const 
std::shared_ptr ) override;
+  void onSchedule(const std::shared_ptr& context, const 
std::shared_ptr& 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 , const 
std::shared_ptr , const 
std::shared_ptr ,
- const std::shared_ptr , bool 
is_success, int64_t status_code);
-  bool shouldEmitFlowFile() const;
+  void route(const std::shared_ptr& request, const 
std::shared_ptr& response, const 
std::shared_ptr& session,
+ const std::shared_ptr& context, bool 
is_success, int64_t status_code);
+  static bool shouldEmitFlowFile(minifi::extensions::curl::HTTPClient& client);
+  void onTriggerWithClient(const std::shared_ptr& 
context, const std::shared_ptr& session, 
minifi::extensions::curl::HTTPClient& client);
   [[nodiscard]] bool appendHeaders(const core::FlowFile& flow_file, 
/*std::invocable*/ auto append_header);
 
-  std::shared_ptr ssl_context_service_;
-  std::string method_;
-  std::string url_;
-  bool date_header_include_{true};
+
+  void setupMembersFromProperties(const core::ProcessContext& context);
+  std::unique_ptr 
createHTTPClientFromPropertiesAndMembers(const core::ProcessContext& context) 
const;
+
   std::optional attributes_to_send_;
-  std::chrono::milliseconds connect_timeout_ms_{2};
-  std::chrono::milliseconds read_timeout_ms_{2};
-  // attribute in which response body will be added
-  std::string put_attribute_name_;
+
+  std::optional 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 
logger_{core::logging::LoggerFactory::getLogger()};
+  
gsl::not_null>>
 client_queue_ = gsl::make_not_null(
+  
utils::ResourceQueue::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-66ae832b17ef9e3a1f26655cb827e35f9e663eb97f2923c91989c160f9f6R76
 



-- 
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-unsubscr...@nifi.apache.org

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



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

2022-08-25 Thread GitBox


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(true)
-  ->build());
+->withDescription("DEPRECATED. Only kept for backwards compatibility, 
no functionality is included.")
+->withDefaultValue(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(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(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 

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

2022-08-25 Thread GitBox


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(true)
-  ->build());
+->withDescription("DEPRECATED. Only kept for backwards compatibility, 
no functionality is included.")
+->withDefaultValue(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(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(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 

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

2022-08-25 Thread GitBox


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 stop;
-  ByteInputCallback *ptr;
-  size_t pos;
-
-  size_t getPos() {
-std::lock_guard lock(mutex);
-return pos;
-  }
+class HTTPUploadCallback : public ByteInputCallback {
+ public:
+  template
+  explicit HTTPUploadCallback(Args&& ... args) : 
ByteInputCallback(std::forward(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-unsubscr...@nifi.apache.org

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



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

2022-08-25 Thread GitBox


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 client)
+HttpStream::HttpStream(std::shared_ptr 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 lock(mutex_);
 if (!started_) {
-  callback_.ptr = _callback_;
-  callback_.pos = 0;
-  http_client_->setUploadCallback(_);
+  auto callback = std::make_unique();
+  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-unsubscr...@nifi.apache.org

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



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

2022-08-25 Thread GitBox


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 client)
+HttpStream::HttpStream(std::shared_ptr 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-unsubscr...@nifi.apache.org

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



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

2022-08-25 Thread GitBox


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 
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, 
::HTTPRequestResponse::recieve_write);
-  curl_easy_setopt(http_session_, CURLOPT_WRITEDATA, 
static_cast(callbackObj));
+void HTTPClient::setReadCallback(std::unique_ptr 
callback) {
+  read_callback_ = std::move(callback);
+  curl_easy_setopt(http_session_.get(), CURLOPT_WRITEFUNCTION, 
::HTTPRequestResponse::recieve_write);
+  curl_easy_setopt(http_session_.get(), CURLOPT_WRITEDATA, 
static_cast(read_callback_.get()));
 }
 
-void HTTPClient::setUploadCallback(HTTPUploadCallback *callbackObj) {
+void HTTPClient::setUploadCallback(std::unique_ptr 
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, 
::HTTPRequestResponse::send_write);
-  curl_easy_setopt(http_session_, CURLOPT_READDATA, 
static_cast(callbackObj));
-}
-
-void HTTPClient::setSeekFunction(HTTPUploadCallback *callbackObj) {
-  curl_easy_setopt(http_session_, CURLOPT_SEEKDATA, 
static_cast(callbackObj));
-  curl_easy_setopt(http_session_, CURLOPT_SEEKFUNCTION, 
::HTTPRequestResponse::seek_callback);
+  curl_easy_setopt(http_session_.get(), CURLOPT_READFUNCTION, 
::HTTPRequestResponse::send_write);
+  curl_easy_setopt(http_session_.get(), CURLOPT_READDATA, 
static_cast(write_callback_.get()));
+  curl_easy_setopt(http_session_.get(), CURLOPT_SEEKDATA, 
static_cast(write_callback_.get()));
+  curl_easy_setopt(http_session_.get(), CURLOPT_SEEKFUNCTION, 
::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);
 }
 
 

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

2022-08-25 Thread GitBox


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 
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, 
::HTTPRequestResponse::recieve_write);
-  curl_easy_setopt(http_session_, CURLOPT_WRITEDATA, 
static_cast(callbackObj));
+void HTTPClient::setReadCallback(std::unique_ptr 
callback) {
+  read_callback_ = std::move(callback);
+  curl_easy_setopt(http_session_.get(), CURLOPT_WRITEFUNCTION, 
::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-unsubscr...@nifi.apache.org

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



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

2022-08-25 Thread GitBox


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 
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-unsubscr...@nifi.apache.org

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



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

2022-08-25 Thread GitBox


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& filename = 
std::nullopt);
+  void addFormPart(const std::string& content_type, const std::string& name, 
std::unique_ptr form_callback, const 
std::optional& filename);
 
   void forceClose();
 
-  void initialize(const std::string , const std::string url = "", const 
std::shared_ptr 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 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 callback) 
override;
 
-  void setSeekFunction(HTTPUploadCallback *callbackObj) override;
+  virtual void setReadCallback(std::unique_ptr 
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-unsubscr...@nifi.apache.org

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



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

2022-08-25 Thread GitBox


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-unsubscr...@nifi.apache.org

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



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

2022-08-25 Thread GitBox


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> script_engine_q_;
+  std::shared_ptr> 
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-unsubscr...@nifi.apache.org

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



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

2022-08-25 Thread GitBox


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 , const 
std::shared_ptr ) override;
+  void onTrigger(const std::shared_ptr& context, const 
std::shared_ptr& session) override;
   void initialize() override;
-  void onSchedule(const std::shared_ptr , const 
std::shared_ptr ) override;
+  void onSchedule(const std::shared_ptr& context, const 
std::shared_ptr& 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 , const 
std::shared_ptr , const 
std::shared_ptr ,
- const std::shared_ptr , bool 
is_success, int64_t status_code);
-  bool shouldEmitFlowFile() const;
+  void route(const std::shared_ptr& request, const 
std::shared_ptr& response, const 
std::shared_ptr& session,
+ const std::shared_ptr& context, bool 
is_success, int64_t status_code);
+  static bool shouldEmitFlowFile(minifi::extensions::curl::HTTPClient& client);
+  void onTriggerWithClient(const std::shared_ptr& 
context, const std::shared_ptr& session, 
minifi::extensions::curl::HTTPClient& client);
   [[nodiscard]] bool appendHeaders(const core::FlowFile& flow_file, 
/*std::invocable*/ auto append_header);
 
-  std::shared_ptr ssl_context_service_;
-  std::string method_;
-  std::string url_;
-  bool date_header_include_{true};
+
+  void setupMembersFromProperties(const core::ProcessContext& context);
+  std::unique_ptr 
createHTTPClientFromPropertiesAndMembers(const core::ProcessContext& context) 
const;
+
   std::optional attributes_to_send_;
-  std::chrono::milliseconds connect_timeout_ms_{2};
-  std::chrono::milliseconds read_timeout_ms_{2};
-  // attribute in which response body will be added
-  std::string put_attribute_name_;
+
+  std::optional 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 
logger_{core::logging::LoggerFactory::getLogger()};
+  
gsl::not_null>>
 client_queue_ = gsl::make_not_null(
+  
utils::ResourceQueue::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-unsubscr...@nifi.apache.org

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



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

2022-08-25 Thread GitBox


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(true)
-  ->build());
+->withDescription("DEPRECATED. Only kept for backwards compatibility, 
no functionality is included.")
+->withDefaultValue(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(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(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 

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

2022-08-23 Thread GitBox


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 
+#include 
+#include 
+#include 
+#include 
+#include 
+#include 
+#include 
+
+#include "core/logging/Logger.h"
+#include "concurrentqueue.h"
+#include "MinifiConcurrentQueue.h"
+
+namespace org::apache::nifi::minifi::utils {
+
+template
+class ResourceQueue : public 
std::enable_shared_from_this> {

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-unsubscr...@nifi.apache.org

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



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

2022-08-23 Thread GitBox


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 );
+  std::chrono::milliseconds getAbsoluteTimeout() const { return 
3*read_timeout_; }
 
-  HTTPReadCallback content_;
+  utils::HTTPReadCallback content_{std::numeric_limits::max()};
 
   std::shared_ptr ssl_context_service_;
   std::string url_;
-  std::chrono::milliseconds connect_timeout_ms_{3};
-  // read timeout.
-  std::chrono::milliseconds read_timeout_ms_{3};
-  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 username_password_;
+  std::unique_ptr http_session_{nullptr, 
curl_easy_cleanup};
+  std::unique_ptr 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-unsubscr...@nifi.apache.org

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



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

2022-08-23 Thread GitBox


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-unsubscr...@nifi.apache.org

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



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

2022-08-23 Thread GitBox


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 
+#include 
+#include 
+#include 
+#include 
+
+#ifdef WIN32
+#pragma comment(lib, "wldap32.lib" )
+#pragma comment(lib, "crypt32.lib" )
+#pragma comment(lib, "Ws2_32.lib")
+
+#define CURL_STATICLIB
+#include 
+#else
+#include 
+#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 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 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-unsubscr...@nifi.apache.org

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



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

2022-08-23 Thread GitBox


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 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-unsubscr...@nifi.apache.org

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



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

2022-08-23 Thread GitBox


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, 
::HTTPRequestResponse::recieve_write);
-  curl_easy_setopt(http_session_, CURLOPT_WRITEDATA, 
static_cast(callbackObj));
+void HTTPClient::setReadCallback(std::unique_ptr&& 
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-unsubscr...@nifi.apache.org

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



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

2022-08-23 Thread GitBox


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(elapsed.count()), 
static_cast(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(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-unsubscr...@nifi.apache.org

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



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

2022-08-23 Thread GitBox


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 ssl_context_service)
+HTTPClient::HTTPClient(std::string url, 
std::shared_ptr  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& filename) {
+void HTTPClient::addFormPart(const std::string& content_type, const 
std::string& name, std::unique_ptr&& form_callback, 
const std::optional& 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-unsubscr...@nifi.apache.org

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



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

2022-08-23 Thread GitBox


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-unsubscr...@nifi.apache.org

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



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

2022-08-23 Thread GitBox


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-unsubscr...@nifi.apache.org

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



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

2022-08-23 Thread GitBox


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 
+#include 
+#include 
+#include 
+#include 
+
+#ifdef WIN32
+#pragma comment(lib, "wldap32.lib" )
+#pragma comment(lib, "crypt32.lib" )
+#pragma comment(lib, "Ws2_32.lib")
+
+#define CURL_STATICLIB
+#include 
+#else
+#include 
+#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 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 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-unsubscr...@nifi.apache.org

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



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

2022-08-17 Thread GitBox


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 stop;
-  ByteInputCallback *ptr;
-  size_t pos;
+class HTTPUploadCallback : public ByteInputCallback {
+ public:
+  template
+  explicit HTTPUploadCallback(Args&& ... args) : 
ByteInputCallback(std::forward(args)...) {}
 
   size_t getPos() {
-std::lock_guard 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-unsubscr...@nifi.apache.org

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



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

2022-08-16 Thread GitBox


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 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 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 client)
+HttpStream::HttpStream(std::shared_ptr 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 lock(mutex_);
 if (!started_) {
-  callback_.ptr = _callback_;
-  callback_.pos = 0;
-  http_client_->setUploadCallback(_);
+  auto callback = std::make_unique(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(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 client)
+HttpStream::HttpStream(std::shared_ptr 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 = 

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

2022-08-11 Thread GitBox


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 
+#include 
+#include 
+#include 
+#include "CivetServer.h"
+
+namespace org::apache::nifi::minifi::extensions::curl::testing {
+
+namespace details {
+
+class NumberedMethodResponder : public CivetHandler {
+ public:
+  explicit NumberedMethodResponder(std::set>& 
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*>(mg_get_user_connection_data(conn));
+connections_.emplace(*user_connection_data);
+  }
+
+  uint64_t response_id_ = 0;
+  std::set>& connections_;
+};
+
+class ReverseBodyPostHandler : public CivetHandler {
+ public:
+  explicit ReverseBodyPostHandler(std::set>& 
connections) : connections_(connections) {}
+
+  bool handlePost(CivetServer* /*server*/, struct mg_connection* conn) 
override {
+saveConnectionId(conn);
+std::vector 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*>(mg_get_user_connection_data(conn));
+connections_.emplace(*user_connection_data);
+  }
+
+  std::set>& 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(id);
+  return 0;
+};
+
+connection_close = [](const struct mg_connection* conn) -> void {
+  auto user_connection_data = 
reinterpret_cast*>(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, 
_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-unsubscr...@nifi.apache.org

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



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

2022-08-11 Thread GitBox


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-unsubscr...@nifi.apache.org

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