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/12/13 12:03:41 UTC

[GitHub] [nifi-minifi-cpp] martinzink opened a new pull request, #1473: MINIFICPP-1973 Refactor ResourceQueue

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

   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:
   - [x] Is there a JIRA ticket associated with this PR? Is it referenced
        in the commit message?
   
   - [x] 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.
   
   - [x] Has your PR been rebased against the latest commit within the target branch (typically main)?
   
   - [x] Is your initial contribution a single, squashed commit?
   
   ### For code changes:
   - [x] 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)?
   - [x] If applicable, have you updated the LICENSE file?
   - [x] If applicable, have you updated the NOTICE file?
   
   ### For documentation related changes:
   - [x] Have you ensured that format looks appropriate for the output in which it is rendered?
   
   ### Note:
   Please ensure that once the PR is submitted, you check GitHub Actions CI results for build issues and submit an update to your PR as soon as possible.
   


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

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

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


[GitHub] [nifi-minifi-cpp] martinzink commented on a diff in pull request #1473: MINIFICPP-1973 Refactor ResourceQueue

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


##########
extensions/splunk/PutSplunkHTTP.cpp:
##########
@@ -40,32 +40,26 @@ void PutSplunkHTTP::initialize() {
   setSupportedRelationships(relationships());
 }
 
-void PutSplunkHTTP::onSchedule(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSessionFactory>& sessionFactory) {
-  SplunkHECProcessor::onSchedule(context, sessionFactory);
-  client_queue_ = utils::ResourceQueue<extensions::curl::HTTPClient>::create(getMaxConcurrentTasks(), logger_);
-}
-
 namespace {
 std::optional<std::string> getContentType(core::ProcessContext& context, const core::FlowFile& flow_file) {
   return context.getProperty(PutSplunkHTTP::ContentType) | utils::orElse ([&flow_file] {return flow_file.getAttribute("mime.type");});
 }
 
-std::string getEndpoint(core::ProcessContext& context, const gsl::not_null<std::shared_ptr<core::FlowFile>>& flow_file, curl::HTTPClient& client) {
+std::string getEndpoint(core::ProcessContext& context, curl::HTTPClient& client) {
   std::stringstream endpoint;
   endpoint << "/services/collector/raw";
   std::vector<std::string> parameters;
-  std::string prop_value;
-  if (context.getProperty(PutSplunkHTTP::SourceType, prop_value, flow_file)) {
-    parameters.push_back("sourcetype=" + client.escape(prop_value));
+  if (auto source_type = context.getProperty(PutSplunkHTTP::SourceType)) {
+    parameters.push_back("sourcetype=" + client.escape(*source_type));
   }
-  if (context.getProperty(PutSplunkHTTP::Source, prop_value, flow_file)) {
-    parameters.push_back("source=" + client.escape(prop_value));
+  if (auto source = context.getProperty(PutSplunkHTTP::Source)) {
+    parameters.push_back("source=" + client.escape(*source));
   }
-  if (context.getProperty(PutSplunkHTTP::Host, prop_value, flow_file)) {
-    parameters.push_back("host=" + client.escape(prop_value));
+  if (auto host = context.getProperty(PutSplunkHTTP::Host)) {
+    parameters.push_back("host=" + client.escape(*host));
   }
-  if (context.getProperty(PutSplunkHTTP::Index, prop_value, flow_file)) {
-    parameters.push_back("index=" + client.escape(prop_value));
+  if (auto index = context.getProperty(PutSplunkHTTP::Index)) {
+    parameters.push_back("index=" + client.escape(*index));

Review Comment:
   Yes, these parameters influence the url which we are uploading to and we only have a single connection so it shouldnt depend on the incoming flowfile only the properties of the 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


[GitHub] [nifi-minifi-cpp] szaszm commented on a diff in pull request #1473: MINIFICPP-1973 Refactor ResourceQueue

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


##########
libminifi/include/utils/ResourceQueue.h:
##########
@@ -126,7 +137,10 @@ struct ResourceQueue<ResourceType>::make_shared_enabler : public ResourceQueue<R
 };
 
 template<class ResourceType>
-auto ResourceQueue<ResourceType>::create(std::optional<size_t> maximum_number_of_creatable_resources, std::shared_ptr<core::logging::Logger> logger) {
-  return std::make_shared<make_shared_enabler>(maximum_number_of_creatable_resources, std::move(logger));
+auto ResourceQueue<ResourceType>::create(std::function<std::unique_ptr<ResourceType>()> creator,
+                                         std::optional<size_t> maximum_number_of_creatable_resources,
+                                         std::optional<std::function<void(ResourceType&)>> reset_fn,
+                                         std::shared_ptr<core::logging::Logger> logger) {
+  return std::make_shared<make_shared_enabler>(std::move(creator), maximum_number_of_creatable_resources, std::move(reset_fn), std::move(logger));

Review Comment:
   This is not a new change, but in `make_shared_enabler`, the forwarding constructor is unnecessary complexity. The same can be achieved by adding this in place of line 135-136:
   ```cpp
       using ResourceQueue<ResourceType>::ResourceQueue;
   ```



##########
extensions/splunk/PutSplunkHTTP.cpp:
##########
@@ -40,32 +40,26 @@ void PutSplunkHTTP::initialize() {
   setSupportedRelationships(relationships());
 }
 
-void PutSplunkHTTP::onSchedule(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSessionFactory>& sessionFactory) {
-  SplunkHECProcessor::onSchedule(context, sessionFactory);
-  client_queue_ = utils::ResourceQueue<extensions::curl::HTTPClient>::create(getMaxConcurrentTasks(), logger_);
-}
-
 namespace {
 std::optional<std::string> getContentType(core::ProcessContext& context, const core::FlowFile& flow_file) {
   return context.getProperty(PutSplunkHTTP::ContentType) | utils::orElse ([&flow_file] {return flow_file.getAttribute("mime.type");});
 }
 
-std::string getEndpoint(core::ProcessContext& context, const gsl::not_null<std::shared_ptr<core::FlowFile>>& flow_file, curl::HTTPClient& client) {
+std::string getEndpoint(core::ProcessContext& context, curl::HTTPClient& client) {
   std::stringstream endpoint;
   endpoint << "/services/collector/raw";
   std::vector<std::string> parameters;
-  std::string prop_value;
-  if (context.getProperty(PutSplunkHTTP::SourceType, prop_value, flow_file)) {
-    parameters.push_back("sourcetype=" + client.escape(prop_value));
+  if (auto source_type = context.getProperty(PutSplunkHTTP::SourceType)) {
+    parameters.push_back("sourcetype=" + client.escape(*source_type));
   }
-  if (context.getProperty(PutSplunkHTTP::Source, prop_value, flow_file)) {
-    parameters.push_back("source=" + client.escape(prop_value));
+  if (auto source = context.getProperty(PutSplunkHTTP::Source)) {
+    parameters.push_back("source=" + client.escape(*source));
   }
-  if (context.getProperty(PutSplunkHTTP::Host, prop_value, flow_file)) {
-    parameters.push_back("host=" + client.escape(prop_value));
+  if (auto host = context.getProperty(PutSplunkHTTP::Host)) {
+    parameters.push_back("host=" + client.escape(*host));
   }
-  if (context.getProperty(PutSplunkHTTP::Index, prop_value, flow_file)) {
-    parameters.push_back("index=" + client.escape(prop_value));
+  if (auto index = context.getProperty(PutSplunkHTTP::Index)) {
+    parameters.push_back("index=" + client.escape(*index));

Review Comment:
   This change causes these properties to no longer support expression language. Is this intentional?



##########
libminifi/test/unit/ResourceQueueTests.cpp:
##########
@@ -76,20 +81,39 @@ TEST_CASE("Resource limitation is not set to the resource queue", "[utils::Resou
 }
 
 TEST_CASE("resource returns when it goes out of scope", "[utils::ResourceQueue]") {
-  auto queue = utils::ResourceQueue<int>::create(std::nullopt, nullptr);
+  using std::chrono::steady_clock;
+  auto queue = utils::ResourceQueue<steady_clock::time_point>::create([]{ return std::make_unique<steady_clock::time_point>(steady_clock::time_point::min()); });
+  {
+    auto resource = queue->getResource();
+    CHECK(*resource == steady_clock::time_point::min());
+    *resource = steady_clock::now();
+  }
+  {
+    auto resource = queue->getResource();
+    CHECK(*resource != steady_clock::time_point::min());
+  }
+}
+
+TEST_CASE("resource resets when it goes out of scope", "[utils::ResourceQueue]") {
+  using std::chrono::steady_clock;
+  auto queue = utils::ResourceQueue<steady_clock::time_point>::create([]{ return std::make_unique<steady_clock::time_point>(steady_clock::time_point::min()); },
+                                                                      std::nullopt,
+                                                                      [](steady_clock::time_point& resource){ resource = steady_clock::time_point::max();});

Review Comment:
   A better way to test this would be to have the reset function assign the same initial value as the create function, but change the value in the first scope, and check that it's back to the default in the second.
   This way you're testing the requirements instead of the implementation.



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

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

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


[GitHub] [nifi-minifi-cpp] fgerlits closed pull request #1473: MINIFICPP-1973 Refactor ResourceQueue

Posted by GitBox <gi...@apache.org>.
fgerlits closed pull request #1473: MINIFICPP-1973 Refactor ResourceQueue
URL: https://github.com/apache/nifi-minifi-cpp/pull/1473


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

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

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


[GitHub] [nifi-minifi-cpp] martinzink commented on a diff in pull request #1473: MINIFICPP-1973 Refactor ResourceQueue

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


##########
libminifi/test/unit/ResourceQueueTests.cpp:
##########
@@ -76,20 +81,39 @@ TEST_CASE("Resource limitation is not set to the resource queue", "[utils::Resou
 }
 
 TEST_CASE("resource returns when it goes out of scope", "[utils::ResourceQueue]") {
-  auto queue = utils::ResourceQueue<int>::create(std::nullopt, nullptr);
+  using std::chrono::steady_clock;
+  auto queue = utils::ResourceQueue<steady_clock::time_point>::create([]{ return std::make_unique<steady_clock::time_point>(steady_clock::time_point::min()); });
+  {
+    auto resource = queue->getResource();
+    CHECK(*resource == steady_clock::time_point::min());
+    *resource = steady_clock::now();
+  }
+  {
+    auto resource = queue->getResource();
+    CHECK(*resource != steady_clock::time_point::min());
+  }
+}
+
+TEST_CASE("resource resets when it goes out of scope", "[utils::ResourceQueue]") {
+  using std::chrono::steady_clock;
+  auto queue = utils::ResourceQueue<steady_clock::time_point>::create([]{ return std::make_unique<steady_clock::time_point>(steady_clock::time_point::min()); },
+                                                                      std::nullopt,
+                                                                      [](steady_clock::time_point& resource){ resource = steady_clock::time_point::max();});

Review Comment:
   Good idea, I've changed this and also included some additional checks aswell in https://github.com/apache/nifi-minifi-cpp/pull/1473/commits/5279824fbde23389279cce46b2859f979d93044f



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

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

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


[GitHub] [nifi-minifi-cpp] szaszm commented on a diff in pull request #1473: MINIFICPP-1973 Refactor ResourceQueue

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


##########
libminifi/include/utils/ResourceQueue.h:
##########
@@ -126,7 +137,10 @@ struct ResourceQueue<ResourceType>::make_shared_enabler : public ResourceQueue<R
 };
 
 template<class ResourceType>
-auto ResourceQueue<ResourceType>::create(std::optional<size_t> maximum_number_of_creatable_resources, std::shared_ptr<core::logging::Logger> logger) {
-  return std::make_shared<make_shared_enabler>(maximum_number_of_creatable_resources, std::move(logger));
+auto ResourceQueue<ResourceType>::create(std::function<std::unique_ptr<ResourceType>()> creator,
+                                         std::optional<size_t> maximum_number_of_creatable_resources,
+                                         std::optional<std::function<void(ResourceType&)>> reset_fn,
+                                         std::shared_ptr<core::logging::Logger> logger) {
+  return std::make_shared<make_shared_enabler>(std::move(creator), maximum_number_of_creatable_resources, std::move(reset_fn), std::move(logger));

Review Comment:
   My bad, thanks for the clarification.



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

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

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


[GitHub] [nifi-minifi-cpp] martinzink commented on a diff in pull request #1473: MINIFICPP-1973 Refactor ResourceQueue

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


##########
libminifi/include/utils/ResourceQueue.h:
##########
@@ -126,7 +137,10 @@ struct ResourceQueue<ResourceType>::make_shared_enabler : public ResourceQueue<R
 };
 
 template<class ResourceType>
-auto ResourceQueue<ResourceType>::create(std::optional<size_t> maximum_number_of_creatable_resources, std::shared_ptr<core::logging::Logger> logger) {
-  return std::make_shared<make_shared_enabler>(maximum_number_of_creatable_resources, std::move(logger));
+auto ResourceQueue<ResourceType>::create(std::function<std::unique_ptr<ResourceType>()> creator,
+                                         std::optional<size_t> maximum_number_of_creatable_resources,
+                                         std::optional<std::function<void(ResourceType&)>> reset_fn,
+                                         std::shared_ptr<core::logging::Logger> logger) {
+  return std::make_shared<make_shared_enabler>(std::move(creator), maximum_number_of_creatable_resources, std::move(reset_fn), std::move(logger));

Review Comment:
   Unfortunetly this cannot be simplified due to
   
   > According to the C++17 Standard (10.3.3 The using declaration):
   >     19 A synonym created by a using-declaration has the usual accessibility for a member-declaration. A using-declarator that names a constructor does not create a synonym; instead, the additional constructors are accessible if they would be accessible when used to construct an object of the corresponding base class, and the accessibility of the using-declaration is ignored.
   
   
   I've also created a smaller example in godbolt to double check it
   https://godbolt.org/z/rfzsPcT6s



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