You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by GitBox <gi...@apache.org> on 2022/08/20 02:04:56 UTC

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

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


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

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



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

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



##########
extensions/http-curl/client/HTTPClient.cpp:
##########
@@ -164,212 +161,191 @@ bool HTTPClient::setSpecificSSLVersion(SSLVersion specific_version) {
 }
 
 // If not set, the default will be TLS 1.0, see https://curl.haxx.se/libcurl/c/CURLOPT_SSLVERSION.html
-bool HTTPClient::setMinimumSSLVersion(SSLVersion minimum_version) {
+bool HTTPClient::setMinimumSSLVersion(utils::SSLVersion minimum_version) {
   CURLcode ret = CURLE_UNKNOWN_OPTION;
   switch (minimum_version) {
-    case SSLVersion::TLSv1_0:
-      ret = curl_easy_setopt(http_session_, CURLOPT_SSLVERSION, CURL_SSLVERSION_TLSv1_0);
+    case utils::SSLVersion::TLSv1_0:
+      ret = curl_easy_setopt(http_session_.get(), CURLOPT_SSLVERSION, CURL_SSLVERSION_TLSv1_0);
       break;
-    case SSLVersion::TLSv1_1:
-      ret = curl_easy_setopt(http_session_, CURLOPT_SSLVERSION, CURL_SSLVERSION_TLSv1_1);
+    case utils::SSLVersion::TLSv1_1:
+      ret = curl_easy_setopt(http_session_.get(), CURLOPT_SSLVERSION, CURL_SSLVERSION_TLSv1_1);
       break;
-    case SSLVersion::TLSv1_2:
-      ret = curl_easy_setopt(http_session_, CURLOPT_SSLVERSION, CURL_SSLVERSION_TLSv1_2);
+    case utils::SSLVersion::TLSv1_2:
+      ret = curl_easy_setopt(http_session_.get(), CURLOPT_SSLVERSION, CURL_SSLVERSION_TLSv1_2);
       break;
   }
 
   return ret == CURLE_OK;
 }
 
-DEPRECATED(/*deprecated in*/ 0.8.0, /*will remove in */ 2.0) void HTTPClient::setConnectionTimeout(int64_t timeout) {
-  setConnectionTimeout(std::chrono::milliseconds(timeout * 1000));
-}
-
-DEPRECATED(/*deprecated in*/ 0.8.0, /*will remove in */ 2.0) void HTTPClient::setReadTimeout(int64_t timeout) {
-  setReadTimeout(std::chrono::milliseconds(timeout * 1000));
-}
-
 void HTTPClient::setConnectionTimeout(std::chrono::milliseconds timeout) {
-  connect_timeout_ms_ = timeout;
+  if (timeout < 0ms) {
+    logger_->log_error("Invalid timeout");
+    return;
+  }
+  connect_timeout_ = timeout;
 }
 
 void HTTPClient::setReadTimeout(std::chrono::milliseconds timeout) {
-  read_timeout_ms_ = timeout;
+  if (timeout < 0ms) {
+    logger_->log_error("Invalid timeout");
+    return;
+  }
+  read_timeout_ = timeout;
 }
 
-void HTTPClient::setReadCallback(HTTPReadCallback *callbackObj) {
-  callback = callbackObj;
-  curl_easy_setopt(http_session_, CURLOPT_WRITEFUNCTION, &utils::HTTPRequestResponse::recieve_write);
-  curl_easy_setopt(http_session_, CURLOPT_WRITEDATA, static_cast<void*>(callbackObj));
+void HTTPClient::setReadCallback(std::unique_ptr<utils::HTTPReadCallback>&& callback) {

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



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

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



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

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



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

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



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

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



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

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



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

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



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

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



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

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

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