You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by "lordgamez (via GitHub)" <gi...@apache.org> on 2023/10/02 14:08:43 UTC

[PR] MINIFICPP-2239 Add bandwidth limit to InvokeHTTP processor [nifi-minifi-cpp]

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

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


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

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

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


Re: [PR] MINIFICPP-2239 Add bandwidth limit to InvokeHTTP processor [nifi-minifi-cpp]

Posted by "szaszm (via GitHub)" <gi...@apache.org>.
szaszm commented on code in PR #1674:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1674#discussion_r1348807865


##########
extensions/http-curl/processors/InvokeHTTP.h:
##########
@@ -177,8 +177,20 @@ class InvokeHTTP : public core::Processor {
       .withDefaultValue(magic_enum::enum_name(invoke_http::InvalidHTTPHeaderFieldHandlingOption::transform))
       .withAllowedValues(magic_enum::enum_names<invoke_http::InvalidHTTPHeaderFieldHandlingOption>())
       .build();
+  EXTENSIONAPI static constexpr auto UploadBandwidthLimit = core::PropertyDefinitionBuilder<>::createProperty("Upload Bandwidth Limit")
+      .withDescription("Maximum data per second to send, use 0 for unlimited.")
+      .withPropertyType(core::StandardPropertyTypes::DATA_SIZE_TYPE)
+      .withDefaultValue("0 B")
+      .isRequired(true)
+      .build();
+  EXTENSIONAPI static constexpr auto DownloadBandwidthLimit = core::PropertyDefinitionBuilder<>::createProperty("Download Bandwidth Limit")

Review Comment:
   I'd prefer to keep referring to "download speed" and "upload speed" in the property name.
   
   Another alternative approach would be to use a custom property type, with no corresponding C2/NiFi validator, and just parse the proper data rate units in onSchedule. I'd like that the most, but it requires a bit more additional code.



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


Re: [PR] MINIFICPP-2239 Add bandwidth limit to InvokeHTTP processor [nifi-minifi-cpp]

Posted by "szaszm (via GitHub)" <gi...@apache.org>.
szaszm commented on code in PR #1674:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1674#discussion_r1355153756


##########
PROCESSORS.md:
##########
@@ -1320,6 +1320,8 @@ In the list below, the names of required properties appear in bold. Any other pr
 | Always Output Response                          | false                    | true<br/>false              | Will force a response FlowFile to be generated and routed to the 'Response' relationship regardless of what the server status code received is                                                                                                                                                                  |
 | Penalize on "No Retry"                          | false                    | true<br/>false              | Enabling this property will penalize FlowFiles that are routed to the "No Retry" relationship.                                                                                                                                                                                                                  |
 | **Invalid HTTP Header Field Handling Strategy** | transform                | fail<br/>transform<br/>drop | Indicates what should happen when an attribute's name is not a valid HTTP header field name. Options: transform - invalid characters are replaced, fail - flow file is transferred to failure, drop - drops invalid attributes from HTTP message                                                                |
+| Upload Speed Limit                              |                          |                             | Maximum data per second to send (e.g. '500 KB/s'). Leave this empty if you want no limit.                                                                                                                                                                                                                       |
+| Download Speed Limit                            |                          |                             | Maximum data per second to receive (e.g. '500 KB/s'). Leave this empty if you want no limit.                                                                                                                                                                                                                    |

Review Comment:
   ```suggestion
   | Upload Speed Limit                              |                          |                             | Maximum upload speed, e.g. '500 KB/s'. Leave this empty if you want no limit.                                                                                                                                                                                                                       |
   | Download Speed Limit                            |                          |                             | Maximum download speed,e.g. '500 KB/s'. Leave this empty if you want no limit.                                                                                                                                                                                                                    |
   ```



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


Re: [PR] MINIFICPP-2239 Add bandwidth limit to InvokeHTTP processor [nifi-minifi-cpp]

Posted by "szaszm (via GitHub)" <gi...@apache.org>.
szaszm commented on code in PR #1674:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1674#discussion_r1345488607


##########
libminifi/src/core/Processor.cpp:
##########
@@ -345,15 +345,15 @@ void Processor::validateAnnotations() const {
   switch (getInputRequirement()) {
     case annotation::Input::INPUT_REQUIRED: {
       if (!hasIncomingConnections()) {
-        throw Exception(PROCESS_SCHEDULE_EXCEPTION, "INPUT_REQUIRED was specified for the processor, but no incoming connections were found");
+        throw Exception(PROCESS_SCHEDULE_EXCEPTION, "INPUT_REQUIRED was specified for processor '" + getName()  + "' (uuid: '" + getUUIDStr() + "'), but no incoming connections were found");

Review Comment:
   I also suggest not removing the "the" from before the word "processor"



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


Re: [PR] MINIFICPP-2239 Add bandwidth limit to InvokeHTTP processor [nifi-minifi-cpp]

Posted by "szaszm (via GitHub)" <gi...@apache.org>.
szaszm commented on code in PR #1674:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1674#discussion_r1344682066


##########
extensions/http-curl/client/HTTPClient.h:
##########
@@ -149,6 +149,8 @@ class HTTPClient : public utils::BaseHTTPClient, public core::Connectable {
 
   void setFollowRedirects(bool follow);
 
+  void setMaximumUploadBandwidth(uint64_t max_bandwidth);
+  void setMaximumDownloadBandwidth(uint64_t max_bandwidth);

Review Comment:
   This needs a unit throughout the interface. A suffix in the identifiers is good enough, with a comment somewhere. Is it Bps?



##########
libminifi/src/core/Processor.cpp:
##########
@@ -345,15 +345,15 @@ void Processor::validateAnnotations() const {
   switch (getInputRequirement()) {
     case annotation::Input::INPUT_REQUIRED: {
       if (!hasIncomingConnections()) {
-        throw Exception(PROCESS_SCHEDULE_EXCEPTION, "INPUT_REQUIRED was specified for the processor, but no incoming connections were found");
+        throw Exception(PROCESS_SCHEDULE_EXCEPTION, "INPUT_REQUIRED was specified for processor '" + getName()  + "' (uuid: '" + getUUIDStr() + "'), but no incoming connections were found");

Review Comment:
   ```suggestion
           throw Exception(PROCESS_SCHEDULE_EXCEPTION, fmt::format("INPUT_REQUIRED was specified for the processor '{}' (uuid: '{}'), but no incoming connections were found", getName(), getUUIDStr()));
   ```



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


Re: [PR] MINIFICPP-2239 Add bandwidth limit to InvokeHTTP processor [nifi-minifi-cpp]

Posted by "lordgamez (via GitHub)" <gi...@apache.org>.
lordgamez commented on code in PR #1674:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1674#discussion_r1356398691


##########
libminifi/include/core/PropertyType.h:
##########
@@ -319,6 +319,24 @@ class TimePeriodPropertyType : public PropertyType {
   }
 };
 
+class DataTransferSpeedPropertyType : public PropertyType {
+ public:
+  constexpr ~DataTransferSpeedPropertyType() override {}  // NOLINT see comment at parent
+
+  [[nodiscard]] std::string_view getValidatorName() const override { return "DATA_SIZE_VALIDATOR"; }

Review Comment:
   It seems that some C2 servers do not ignore the new validator, but return on error on it, so I removed the validator for this type in 2ec3285238625ebd7348f517d14cf6840f456351



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


Re: [PR] MINIFICPP-2239 Add bandwidth limit to InvokeHTTP processor [nifi-minifi-cpp]

Posted by "martinzink (via GitHub)" <gi...@apache.org>.
martinzink commented on code in PR #1674:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1674#discussion_r1348424846


##########
extensions/http-curl/processors/InvokeHTTP.h:
##########
@@ -177,8 +177,20 @@ class InvokeHTTP : public core::Processor {
       .withDefaultValue(magic_enum::enum_name(invoke_http::InvalidHTTPHeaderFieldHandlingOption::transform))
       .withAllowedValues(magic_enum::enum_names<invoke_http::InvalidHTTPHeaderFieldHandlingOption>())
       .build();
+  EXTENSIONAPI static constexpr auto UploadBandwidthLimit = core::PropertyDefinitionBuilder<>::createProperty("Upload Bandwidth Limit")
+      .withDescription("Maximum data per second to send, use 0 for unlimited.")
+      .withPropertyType(core::StandardPropertyTypes::DATA_SIZE_TYPE)
+      .withDefaultValue("0 B")
+      .isRequired(true)
+      .build();
+  EXTENSIONAPI static constexpr auto DownloadBandwidthLimit = core::PropertyDefinitionBuilder<>::createProperty("Download Bandwidth Limit")

Review Comment:
   Im a bit torn on this because the current problem is that the dimension of the property doesnt match its name. 
   Ideally to a "speed" we would accept something like 100 KBps not 100 KB, but that would require a much larger change, maybe we could  rename it so its name at least matches the expected unit e.g. "Upload data limit per seconds"
   
   What do you guys think? @lordgamez  @szaszm 



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


Re: [PR] MINIFICPP-2239 Add bandwidth limit to InvokeHTTP processor [nifi-minifi-cpp]

Posted by "szaszm (via GitHub)" <gi...@apache.org>.
szaszm commented on code in PR #1674:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1674#discussion_r1345801824


##########
extensions/http-curl/processors/InvokeHTTP.h:
##########
@@ -177,8 +177,20 @@ class InvokeHTTP : public core::Processor {
       .withDefaultValue(magic_enum::enum_name(invoke_http::InvalidHTTPHeaderFieldHandlingOption::transform))
       .withAllowedValues(magic_enum::enum_names<invoke_http::InvalidHTTPHeaderFieldHandlingOption>())
       .build();
+  EXTENSIONAPI static constexpr auto UploadBandwidthLimit = core::PropertyDefinitionBuilder<>::createProperty("Upload Bandwidth Limit")
+      .withDescription("Maximum data per second to send, use 0 for unlimited.")
+      .withPropertyType(core::StandardPropertyTypes::DATA_SIZE_TYPE)
+      .withDefaultValue("0 B")
+      .isRequired(true)
+      .build();
+  EXTENSIONAPI static constexpr auto DownloadBandwidthLimit = core::PropertyDefinitionBuilder<>::createProperty("Download Bandwidth Limit")

Review Comment:
   I would use the world "speed" instead of "bandwidth".
   
   If we want to be precise, the word "bandwidth" for general network transfer speeds is inaccurate: modern networks can transfer much more data even on narrower bands than older protocols. In everyday speech, in the context of internet speed, the two words are used as synonyms, but in this context, I think "speed" is clearer.



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


Re: [PR] MINIFICPP-2239 Add bandwidth limit to InvokeHTTP processor [nifi-minifi-cpp]

Posted by "lordgamez (via GitHub)" <gi...@apache.org>.
lordgamez commented on code in PR #1674:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1674#discussion_r1355279283


##########
docker/test/integration/features/MiNiFi_integration_test_driver.py:
##########
@@ -244,11 +246,21 @@ def check_for_num_files_generated(self, num_flowfiles, timeout_seconds):
         output_validator.set_output_dir(self.file_system_observer.get_output_dir())
         self.__check_output(timeout_seconds, output_validator, max(1, num_flowfiles))
 
-    def check_for_num_file_range_generated(self, min_files, max_files, wait_time_in_seconds):
+    def check_for_num_file_range_generated_after_timeout(self, min_files: int, max_files: int, wait_time_in_seconds: int):
         output_validator = NumFileRangeValidator(min_files, max_files)
         output_validator.set_output_dir(self.file_system_observer.get_output_dir())
         self.__check_output_after_time_period(wait_time_in_seconds, output_validator)
 
+    def check_for_num_file_range_generated(self, min_files: int, max_files: int, wait_time_in_seconds: int):
+        output_validator = NumFileRangeValidator(min_files, max_files)
+        output_validator.set_output_dir(self.file_system_observer.get_output_dir())
+        self.__check_output_over_time_period(wait_time_in_seconds, output_validator)

Review Comment:
   Updated in 41364556e36680ca7f8d564ffa08fe110188a34c



##########
extensions/http-curl/processors/InvokeHTTP.cpp:
##########
@@ -115,6 +115,8 @@ void InvokeHTTP::setupMembersFromProperties(const core::ProcessContext& context)
 
   use_chunked_encoding_ = (context.getProperty(UseChunkedEncoding) | utils::andThen(&utils::StringUtils::toBool)).value_or(false);
   send_date_header_ = context.getProperty<bool>(DateHeader).value_or(true);
+  context.getProperty(UploadSpeedLimit, maximum_upload_speed_);
+  context.getProperty(DownloadSpeedLimit, maximum_download_speed_);

Review Comment:
   Good catch, updated in 41364556e36680ca7f8d564ffa08fe110188a34c



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


Re: [PR] MINIFICPP-2239 Add bandwidth limit to InvokeHTTP processor [nifi-minifi-cpp]

Posted by "szaszm (via GitHub)" <gi...@apache.org>.
szaszm closed pull request #1674: MINIFICPP-2239 Add bandwidth limit to InvokeHTTP processor
URL: https://github.com/apache/nifi-minifi-cpp/pull/1674


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


Re: [PR] MINIFICPP-2239 Add bandwidth limit to InvokeHTTP processor [nifi-minifi-cpp]

Posted by "martinzink (via GitHub)" <gi...@apache.org>.
martinzink commented on code in PR #1674:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1674#discussion_r1348617306


##########
extensions/http-curl/processors/InvokeHTTP.h:
##########
@@ -177,8 +177,20 @@ class InvokeHTTP : public core::Processor {
       .withDefaultValue(magic_enum::enum_name(invoke_http::InvalidHTTPHeaderFieldHandlingOption::transform))
       .withAllowedValues(magic_enum::enum_names<invoke_http::InvalidHTTPHeaderFieldHandlingOption>())
       .build();
+  EXTENSIONAPI static constexpr auto UploadBandwidthLimit = core::PropertyDefinitionBuilder<>::createProperty("Upload Bandwidth Limit")
+      .withDescription("Maximum data per second to send, use 0 for unlimited.")
+      .withPropertyType(core::StandardPropertyTypes::DATA_SIZE_TYPE)
+      .withDefaultValue("0 B")
+      .isRequired(true)
+      .build();
+  EXTENSIONAPI static constexpr auto DownloadBandwidthLimit = core::PropertyDefinitionBuilder<>::createProperty("Download Bandwidth Limit")

Review Comment:
   I like the "Upload Speed Limit (/sec)" naming



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


Re: [PR] MINIFICPP-2239 Add bandwidth limit to InvokeHTTP processor [nifi-minifi-cpp]

Posted by "lordgamez (via GitHub)" <gi...@apache.org>.
lordgamez commented on code in PR #1674:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1674#discussion_r1345503109


##########
libminifi/src/core/Processor.cpp:
##########
@@ -345,15 +345,15 @@ void Processor::validateAnnotations() const {
   switch (getInputRequirement()) {
     case annotation::Input::INPUT_REQUIRED: {
       if (!hasIncomingConnections()) {
-        throw Exception(PROCESS_SCHEDULE_EXCEPTION, "INPUT_REQUIRED was specified for the processor, but no incoming connections were found");
+        throw Exception(PROCESS_SCHEDULE_EXCEPTION, "INPUT_REQUIRED was specified for processor '" + getName()  + "' (uuid: '" + getUUIDStr() + "'), but no incoming connections were found");

Review Comment:
   Updated in c8dd0743843bb2b2fd505113ee66693ecf1537df



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


Re: [PR] MINIFICPP-2239 Add bandwidth limit to InvokeHTTP processor [nifi-minifi-cpp]

Posted by "szaszm (via GitHub)" <gi...@apache.org>.
szaszm commented on code in PR #1674:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1674#discussion_r1348602186


##########
extensions/http-curl/processors/InvokeHTTP.h:
##########
@@ -177,8 +177,20 @@ class InvokeHTTP : public core::Processor {
       .withDefaultValue(magic_enum::enum_name(invoke_http::InvalidHTTPHeaderFieldHandlingOption::transform))
       .withAllowedValues(magic_enum::enum_names<invoke_http::InvalidHTTPHeaderFieldHandlingOption>())
       .build();
+  EXTENSIONAPI static constexpr auto UploadBandwidthLimit = core::PropertyDefinitionBuilder<>::createProperty("Upload Bandwidth Limit")
+      .withDescription("Maximum data per second to send, use 0 for unlimited.")
+      .withPropertyType(core::StandardPropertyTypes::DATA_SIZE_TYPE)
+      .withDefaultValue("0 B")
+      .isRequired(true)
+      .build();
+  EXTENSIONAPI static constexpr auto DownloadBandwidthLimit = core::PropertyDefinitionBuilder<>::createProperty("Download Bandwidth Limit")

Review Comment:
   It's not ideal, but I don't think renaming to "upload data limit per second" would make it much better. Technically it's more fitting, but it's still confusing, and the name is less like what a user would look for.
   
   I'd prefer to go with the current version, and if we introduce the necessary units to the ecosystem later, then we can extend the range of accepted values to include the proper unit.
   
   A compromise could be naming it "Upload Speed Limit (/sec)".



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


Re: [PR] MINIFICPP-2239 Add bandwidth limit to InvokeHTTP processor [nifi-minifi-cpp]

Posted by "lordgamez (via GitHub)" <gi...@apache.org>.
lordgamez commented on code in PR #1674:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1674#discussion_r1355001874


##########
PROCESSORS.md:
##########
@@ -1320,6 +1320,8 @@ In the list below, the names of required properties appear in bold. Any other pr
 | Always Output Response                          | false                    | true<br/>false              | Will force a response FlowFile to be generated and routed to the 'Response' relationship regardless of what the server status code received is                                                                                                                                                                  |
 | Penalize on "No Retry"                          | false                    | true<br/>false              | Enabling this property will penalize FlowFiles that are routed to the "No Retry" relationship.                                                                                                                                                                                                                  |
 | **Invalid HTTP Header Field Handling Strategy** | transform                | fail<br/>transform<br/>drop | Indicates what should happen when an attribute's name is not a valid HTTP header field name. Options: transform - invalid characters are replaced, fail - flow file is transferred to failure, drop - drops invalid attributes from HTTP message                                                                |
+| **Upload Speed Limit**                          | 0 B/s                    |                             | Maximum data per second to send, use 0 B/s for unlimited.                                                                                                                                                                                                                                                       |
+| **Download Speed Limit**                        | 0 B/s                    |                             | Maximum data per second to receive, use 0 B/s for unlimited.                                                                                                                                                                                                                                                    |

Review Comment:
   Updated in f8833c3b01c5edf0ff60fff593b2f21ab91763cd



##########
libminifi/include/core/TypedValues.h:
##########
@@ -149,10 +144,39 @@ class DataSizeValue : public TransformableValue, public state::response::UInt64V
   }
 };
 
-}  // namespace core
-}  // namespace minifi
-}  // namespace nifi
-}  // namespace apache
-}  // namespace org
+class DataTransferSpeedValue : public TransformableValue, public state::response::UInt64Value {
+ public:
+  MINIFIAPI static const std::type_index type_id;
+
+  explicit DataTransferSpeedValue(const std::string &transfer_speed_string)
+      : state::response::UInt64Value(0) {
+    DataSizeValue::StringToInt<uint64_t>(removePerSecSuffix(transfer_speed_string), value);
+    string_value = transfer_speed_string;
+  }
+
+  explicit DataTransferSpeedValue(uint64_t value)
+      : state::response::UInt64Value(value) {
+  }
+
+  DataTransferSpeedValue()
+      : state::response::UInt64Value(0) {
+  }
+
+  static std::string removePerSecSuffix(const std::string &input) {
+    auto lower_case_input = utils::StringUtils::toLower(input);
+    if (lower_case_input.ends_with("/sec")) {
+      return input.substr(0, input.size() - 4);
+    }

Review Comment:
   Updated in f8833c3b01c5edf0ff60fff593b2f21ab91763cd



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


Re: [PR] MINIFICPP-2239 Add bandwidth limit to InvokeHTTP processor [nifi-minifi-cpp]

Posted by "lordgamez (via GitHub)" <gi...@apache.org>.
lordgamez commented on code in PR #1674:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1674#discussion_r1356403290


##########
PROCESSORS.md:
##########
@@ -1320,6 +1320,8 @@ In the list below, the names of required properties appear in bold. Any other pr
 | Always Output Response                          | false                    | true<br/>false              | Will force a response FlowFile to be generated and routed to the 'Response' relationship regardless of what the server status code received is                                                                                                                                                                  |
 | Penalize on "No Retry"                          | false                    | true<br/>false              | Enabling this property will penalize FlowFiles that are routed to the "No Retry" relationship.                                                                                                                                                                                                                  |
 | **Invalid HTTP Header Field Handling Strategy** | transform                | fail<br/>transform<br/>drop | Indicates what should happen when an attribute's name is not a valid HTTP header field name. Options: transform - invalid characters are replaced, fail - flow file is transferred to failure, drop - drops invalid attributes from HTTP message                                                                |
+| Upload Speed Limit                              |                          |                             | Maximum data per second to send (e.g. '500 KB/s'). Leave this empty if you want no limit.                                                                                                                                                                                                                       |
+| Download Speed Limit                            |                          |                             | Maximum data per second to receive (e.g. '500 KB/s'). Leave this empty if you want no limit.                                                                                                                                                                                                                    |

Review Comment:
   Updated in d305cce439c0b60fac04b646b79f46f3a4e68d54



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


Re: [PR] MINIFICPP-2239 Add bandwidth limit to InvokeHTTP processor [nifi-minifi-cpp]

Posted by "martinzink (via GitHub)" <gi...@apache.org>.
martinzink commented on code in PR #1674:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1674#discussion_r1348424846


##########
extensions/http-curl/processors/InvokeHTTP.h:
##########
@@ -177,8 +177,20 @@ class InvokeHTTP : public core::Processor {
       .withDefaultValue(magic_enum::enum_name(invoke_http::InvalidHTTPHeaderFieldHandlingOption::transform))
       .withAllowedValues(magic_enum::enum_names<invoke_http::InvalidHTTPHeaderFieldHandlingOption>())
       .build();
+  EXTENSIONAPI static constexpr auto UploadBandwidthLimit = core::PropertyDefinitionBuilder<>::createProperty("Upload Bandwidth Limit")
+      .withDescription("Maximum data per second to send, use 0 for unlimited.")
+      .withPropertyType(core::StandardPropertyTypes::DATA_SIZE_TYPE)
+      .withDefaultValue("0 B")
+      .isRequired(true)
+      .build();
+  EXTENSIONAPI static constexpr auto DownloadBandwidthLimit = core::PropertyDefinitionBuilder<>::createProperty("Download Bandwidth Limit")

Review Comment:
   Im a bit torn on this because the current problem is that the dimension of the property doesnt match its name. 
   Ideally to a "speed" we could accept something like 100 KBps, but that would require a much larger change, maybe we could  rename it so its name at least matches the expected unit e.g. "Upload data limit per seconds"
   
   What do you guys think? @lordgamez  @szaszm 



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


Re: [PR] MINIFICPP-2239 Add bandwidth limit to InvokeHTTP processor [nifi-minifi-cpp]

Posted by "lordgamez (via GitHub)" <gi...@apache.org>.
lordgamez commented on code in PR #1674:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1674#discussion_r1346958713


##########
extensions/http-curl/processors/InvokeHTTP.h:
##########
@@ -177,8 +177,20 @@ class InvokeHTTP : public core::Processor {
       .withDefaultValue(magic_enum::enum_name(invoke_http::InvalidHTTPHeaderFieldHandlingOption::transform))
       .withAllowedValues(magic_enum::enum_names<invoke_http::InvalidHTTPHeaderFieldHandlingOption>())
       .build();
+  EXTENSIONAPI static constexpr auto UploadBandwidthLimit = core::PropertyDefinitionBuilder<>::createProperty("Upload Bandwidth Limit")
+      .withDescription("Maximum data per second to send, use 0 for unlimited.")
+      .withPropertyType(core::StandardPropertyTypes::DATA_SIZE_TYPE)
+      .withDefaultValue("0 B")
+      .isRequired(true)
+      .build();
+  EXTENSIONAPI static constexpr auto DownloadBandwidthLimit = core::PropertyDefinitionBuilder<>::createProperty("Download Bandwidth Limit")

Review Comment:
   Updated in 4f1691020c4bb6b7c39c131d1aa15b2d0f2ecedd



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


Re: [PR] MINIFICPP-2239 Add bandwidth limit to InvokeHTTP processor [nifi-minifi-cpp]

Posted by "lordgamez (via GitHub)" <gi...@apache.org>.
lordgamez commented on code in PR #1674:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1674#discussion_r1348739290


##########
extensions/http-curl/processors/InvokeHTTP.h:
##########
@@ -177,8 +177,20 @@ class InvokeHTTP : public core::Processor {
       .withDefaultValue(magic_enum::enum_name(invoke_http::InvalidHTTPHeaderFieldHandlingOption::transform))
       .withAllowedValues(magic_enum::enum_names<invoke_http::InvalidHTTPHeaderFieldHandlingOption>())
       .build();
+  EXTENSIONAPI static constexpr auto UploadBandwidthLimit = core::PropertyDefinitionBuilder<>::createProperty("Upload Bandwidth Limit")
+      .withDescription("Maximum data per second to send, use 0 for unlimited.")
+      .withPropertyType(core::StandardPropertyTypes::DATA_SIZE_TYPE)
+      .withDefaultValue("0 B")
+      .isRequired(true)
+      .build();
+  EXTENSIONAPI static constexpr auto DownloadBandwidthLimit = core::PropertyDefinitionBuilder<>::createProperty("Download Bandwidth Limit")

Review Comment:
   I think the "per second" is a good idea, maybe we could name them "Data Upload Limit Per Second" and "Data Download Limit Per Second"?



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


Re: [PR] MINIFICPP-2239 Add bandwidth limit to InvokeHTTP processor [nifi-minifi-cpp]

Posted by "fgerlits (via GitHub)" <gi...@apache.org>.
fgerlits commented on code in PR #1674:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1674#discussion_r1355024576


##########
docker/test/integration/features/MiNiFi_integration_test_driver.py:
##########
@@ -244,11 +246,21 @@ def check_for_num_files_generated(self, num_flowfiles, timeout_seconds):
         output_validator.set_output_dir(self.file_system_observer.get_output_dir())
         self.__check_output(timeout_seconds, output_validator, max(1, num_flowfiles))
 
-    def check_for_num_file_range_generated(self, min_files, max_files, wait_time_in_seconds):
+    def check_for_num_file_range_generated_after_timeout(self, min_files: int, max_files: int, wait_time_in_seconds: int):
         output_validator = NumFileRangeValidator(min_files, max_files)
         output_validator.set_output_dir(self.file_system_observer.get_output_dir())
         self.__check_output_after_time_period(wait_time_in_seconds, output_validator)
 
+    def check_for_num_file_range_generated(self, min_files: int, max_files: int, wait_time_in_seconds: int):
+        output_validator = NumFileRangeValidator(min_files, max_files)
+        output_validator.set_output_dir(self.file_system_observer.get_output_dir())
+        self.__check_output_over_time_period(wait_time_in_seconds, output_validator)

Review Comment:
   the difference between these two could be made clearer:
   
   ```suggestion
       def check_for_num_file_range_generated_after_wait(self, min_files: int, max_files: int, wait_time_in_seconds: int):
           output_validator = NumFileRangeValidator(min_files, max_files)
           output_validator.set_output_dir(self.file_system_observer.get_output_dir())
           self.__check_output_after_time_period(wait_time_in_seconds, output_validator)
   
       def check_for_num_file_range_generated_with_timeout(self, min_files: int, max_files: int, timeout_in_seconds: int):
           output_validator = NumFileRangeValidator(min_files, max_files)
           output_validator.set_output_dir(self.file_system_observer.get_output_dir())
           self.__check_output_over_time_period(timeout_in_seconds, output_validator)
   ```



##########
extensions/http-curl/processors/InvokeHTTP.cpp:
##########
@@ -115,6 +115,8 @@ void InvokeHTTP::setupMembersFromProperties(const core::ProcessContext& context)
 
   use_chunked_encoding_ = (context.getProperty(UseChunkedEncoding) | utils::andThen(&utils::StringUtils::toBool)).value_or(false);
   send_date_header_ = context.getProperty<bool>(DateHeader).value_or(true);
+  context.getProperty(UploadSpeedLimit, maximum_upload_speed_);
+  context.getProperty(DownloadSpeedLimit, maximum_download_speed_);

Review Comment:
   These will parse the the properties as numbers, ignoring the suffix, so `Download Speed Limit = 10 MB/s` will result in `maximum_download_speed_ = 10` instead of `maximum_download_speed_ = 10000000`.
   
   If you changed the type of these two fields to `core::DataTransferSpeedValue`, then it would work.  In that case, we would need to call `getValue()` when passing the values to curl.



##########
libminifi/include/core/TypedValues.h:
##########
@@ -149,10 +144,36 @@ class DataSizeValue : public TransformableValue, public state::response::UInt64V
   }
 };
 
-}  // namespace core
-}  // namespace minifi
-}  // namespace nifi
-}  // namespace apache
-}  // namespace org
+class DataTransferSpeedValue : public TransformableValue, public state::response::UInt64Value {
+ public:
+  MINIFIAPI static const std::type_index type_id;
+
+  explicit DataTransferSpeedValue(const std::string &transfer_speed_string)
+      : state::response::UInt64Value(0) {
+    DataSizeValue::StringToInt<uint64_t>(removePerSecSuffix(transfer_speed_string), value);
+    string_value = transfer_speed_string;
+  }
+
+  explicit DataTransferSpeedValue(uint64_t value)
+      : state::response::UInt64Value(value) {
+  }
+
+  DataTransferSpeedValue()
+      : state::response::UInt64Value(0) {
+  }
+
+  static std::string removePerSecSuffix(const std::string &input) {
+    auto lower_case_input = utils::StringUtils::trim(utils::StringUtils::toLower(input));
+    if (lower_case_input.ends_with("/s")) {
+      return lower_case_input.substr(0, lower_case_input.size() - 2);
+    }
+    throw utils::internal::ParseException("Couldn't parse DataTransferSpeedValue, no valid suffix (/s, /sec, ps) found!");

Review Comment:
   this error message is out of date



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


Re: [PR] MINIFICPP-2239 Add bandwidth limit to InvokeHTTP processor [nifi-minifi-cpp]

Posted by "szaszm (via GitHub)" <gi...@apache.org>.
szaszm commented on code in PR #1674:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1674#discussion_r1352413510


##########
extensions/http-curl/processors/InvokeHTTP.h:
##########
@@ -177,8 +177,20 @@ class InvokeHTTP : public core::Processor {
       .withDefaultValue(magic_enum::enum_name(invoke_http::InvalidHTTPHeaderFieldHandlingOption::transform))
       .withAllowedValues(magic_enum::enum_names<invoke_http::InvalidHTTPHeaderFieldHandlingOption>())
       .build();
+  EXTENSIONAPI static constexpr auto UploadBandwidthLimit = core::PropertyDefinitionBuilder<>::createProperty("Upload Bandwidth Limit")
+      .withDescription("Maximum data per second to send, use 0 for unlimited.")
+      .withPropertyType(core::StandardPropertyTypes::DATA_SIZE_TYPE)
+      .withDefaultValue("0 B")
+      .isRequired(true)
+      .build();
+  EXTENSIONAPI static constexpr auto DownloadBandwidthLimit = core::PropertyDefinitionBuilder<>::createProperty("Download Bandwidth Limit")

Review Comment:
   If there are no objections, I suggest we keep the name, and change the format to e.g. "MB/s" meaning mebibytes/sec (to be consistent with data size units), with agent-side validation, and no specific validator in the C2 manifest.



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


Re: [PR] MINIFICPP-2239 Add bandwidth limit to InvokeHTTP processor [nifi-minifi-cpp]

Posted by "lordgamez (via GitHub)" <gi...@apache.org>.
lordgamez commented on code in PR #1674:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1674#discussion_r1345399018


##########
libminifi/src/core/Processor.cpp:
##########
@@ -345,15 +345,15 @@ void Processor::validateAnnotations() const {
   switch (getInputRequirement()) {
     case annotation::Input::INPUT_REQUIRED: {
       if (!hasIncomingConnections()) {
-        throw Exception(PROCESS_SCHEDULE_EXCEPTION, "INPUT_REQUIRED was specified for the processor, but no incoming connections were found");
+        throw Exception(PROCESS_SCHEDULE_EXCEPTION, "INPUT_REQUIRED was specified for processor '" + getName()  + "' (uuid: '" + getUUIDStr() + "'), but no incoming connections were found");

Review Comment:
   Good idea, updated in fc9144144a07100efc52ef0c80a462db7995f38e



##########
extensions/http-curl/client/HTTPClient.h:
##########
@@ -149,6 +149,8 @@ class HTTPClient : public utils::BaseHTTPClient, public core::Connectable {
 
   void setFollowRedirects(bool follow);
 
+  void setMaximumUploadBandwidth(uint64_t max_bandwidth);
+  void setMaximumDownloadBandwidth(uint64_t max_bandwidth);

Review Comment:
   Good point! Yes it is Bps, renamed arguments and added comment in fc9144144a07100efc52ef0c80a462db7995f38e



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


Re: [PR] MINIFICPP-2239 Add bandwidth limit to InvokeHTTP processor [nifi-minifi-cpp]

Posted by "szaszm (via GitHub)" <gi...@apache.org>.
szaszm commented on code in PR #1674:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1674#discussion_r1354915765


##########
libminifi/include/core/TypedValues.h:
##########
@@ -149,10 +144,39 @@ class DataSizeValue : public TransformableValue, public state::response::UInt64V
   }
 };
 
-}  // namespace core
-}  // namespace minifi
-}  // namespace nifi
-}  // namespace apache
-}  // namespace org
+class DataTransferSpeedValue : public TransformableValue, public state::response::UInt64Value {
+ public:
+  MINIFIAPI static const std::type_index type_id;
+
+  explicit DataTransferSpeedValue(const std::string &transfer_speed_string)
+      : state::response::UInt64Value(0) {
+    DataSizeValue::StringToInt<uint64_t>(removePerSecSuffix(transfer_speed_string), value);
+    string_value = transfer_speed_string;
+  }
+
+  explicit DataTransferSpeedValue(uint64_t value)
+      : state::response::UInt64Value(value) {
+  }
+
+  DataTransferSpeedValue()
+      : state::response::UInt64Value(0) {
+  }
+
+  static std::string removePerSecSuffix(const std::string &input) {
+    auto lower_case_input = utils::StringUtils::toLower(input);
+    if (lower_case_input.ends_with("/sec")) {
+      return input.substr(0, input.size() - 4);
+    }

Review Comment:
   I'd prefer to keep the set of accepted formats smaller, and only accept "/s", not "/sec" or "ps". It's easy to extend later, but it harder to break what worked before.



##########
PROCESSORS.md:
##########
@@ -1320,6 +1320,8 @@ In the list below, the names of required properties appear in bold. Any other pr
 | Always Output Response                          | false                    | true<br/>false              | Will force a response FlowFile to be generated and routed to the 'Response' relationship regardless of what the server status code received is                                                                                                                                                                  |
 | Penalize on "No Retry"                          | false                    | true<br/>false              | Enabling this property will penalize FlowFiles that are routed to the "No Retry" relationship.                                                                                                                                                                                                                  |
 | **Invalid HTTP Header Field Handling Strategy** | transform                | fail<br/>transform<br/>drop | Indicates what should happen when an attribute's name is not a valid HTTP header field name. Options: transform - invalid characters are replaced, fail - flow file is transferred to failure, drop - drops invalid attributes from HTTP message                                                                |
+| **Upload Speed Limit**                          | 0 B/s                    |                             | Maximum data per second to send, use 0 B/s for unlimited.                                                                                                                                                                                                                                                       |
+| **Download Speed Limit**                        | 0 B/s                    |                             | Maximum data per second to receive, use 0 B/s for unlimited.                                                                                                                                                                                                                                                    |

Review Comment:
   It would be better to treat an empty value as no limit. We can also keep treating "0 B/s" as no limit, but I'd document the empty as unlimited instead.
   ```suggestion
   | **Upload Speed Limit**                          | 0 B/s                    |                             | Maximum data per second to send. Leave this empty if you want no limit.                                                                                                                                                                                                                                                       |
   | **Download Speed Limit**                        | 0 B/s                    |                             | Maximum data per second to receive. Leave this empty if you want no limit.                                                                                                                                                                                                                                                    |
   ```



##########
libminifi/include/core/PropertyType.h:
##########
@@ -319,6 +319,24 @@ class TimePeriodPropertyType : public PropertyType {
   }
 };
 
+class DataTransferSpeedPropertyType : public PropertyType {
+ public:
+  constexpr ~DataTransferSpeedPropertyType() override {}  // NOLINT see comment at parent
+
+  [[nodiscard]] std::string_view getValidatorName() const override { return "DATA_SIZE_VALIDATOR"; }

Review Comment:
   The DATA_SIZE_VALIDATOR won't work for data transfer speeds. Use VALID_VALIDATOR or similar. The C2 server uses this validator name to select the server-side validation algorithm.



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


Re: [PR] MINIFICPP-2239 Add bandwidth limit to InvokeHTTP processor [nifi-minifi-cpp]

Posted by "lordgamez (via GitHub)" <gi...@apache.org>.
lordgamez commented on code in PR #1674:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1674#discussion_r1355003290


##########
libminifi/include/core/PropertyType.h:
##########
@@ -319,6 +319,24 @@ class TimePeriodPropertyType : public PropertyType {
   }
 };
 
+class DataTransferSpeedPropertyType : public PropertyType {
+ public:
+  constexpr ~DataTransferSpeedPropertyType() override {}  // NOLINT see comment at parent
+
+  [[nodiscard]] std::string_view getValidatorName() const override { return "DATA_SIZE_VALIDATOR"; }

Review Comment:
   Changed to `DATA_TRANSFER_SPEED_VALIDATOR` in f8833c3b01c5edf0ff60fff593b2f21ab91763cd, it should be ignored on C2 server side if not implemented.



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


Re: [PR] MINIFICPP-2239 Add bandwidth limit to InvokeHTTP processor [nifi-minifi-cpp]

Posted by "lordgamez (via GitHub)" <gi...@apache.org>.
lordgamez commented on code in PR #1674:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1674#discussion_r1355279672


##########
libminifi/include/core/TypedValues.h:
##########
@@ -149,10 +144,36 @@ class DataSizeValue : public TransformableValue, public state::response::UInt64V
   }
 };
 
-}  // namespace core
-}  // namespace minifi
-}  // namespace nifi
-}  // namespace apache
-}  // namespace org
+class DataTransferSpeedValue : public TransformableValue, public state::response::UInt64Value {
+ public:
+  MINIFIAPI static const std::type_index type_id;
+
+  explicit DataTransferSpeedValue(const std::string &transfer_speed_string)
+      : state::response::UInt64Value(0) {
+    DataSizeValue::StringToInt<uint64_t>(removePerSecSuffix(transfer_speed_string), value);
+    string_value = transfer_speed_string;
+  }
+
+  explicit DataTransferSpeedValue(uint64_t value)
+      : state::response::UInt64Value(value) {
+  }
+
+  DataTransferSpeedValue()
+      : state::response::UInt64Value(0) {
+  }
+
+  static std::string removePerSecSuffix(const std::string &input) {
+    auto lower_case_input = utils::StringUtils::trim(utils::StringUtils::toLower(input));
+    if (lower_case_input.ends_with("/s")) {
+      return lower_case_input.substr(0, lower_case_input.size() - 2);
+    }
+    throw utils::internal::ParseException("Couldn't parse DataTransferSpeedValue, no valid suffix (/s, /sec, ps) found!");

Review Comment:
   Updated in 41364556e36680ca7f8d564ffa08fe110188a34c



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