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/29 16:19:19 UTC

[GitHub] [nifi-minifi-cpp] lordgamez opened a new pull request, #1411: MINIFICPP-1931 Stabilize ResourceQueueTests

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

   Due to `std::set` not being thread safe, manipulating it in the test from multiple threads could leave it in an inconsistent state. In some cases this resulted in a state where the set contained the 2 correct members, but the `size()` call returned 3.
   
   https://issues.apache.org/jira/browse/MINIFICPP-1931
   
   -----------------------------------
   Thank you for submitting a contribution to Apache NiFi - MiNiFi C++.
   
   In order to streamline the review of the contribution we ask you
   to ensure the following steps have been taken:
   
   ### For all changes:
   - [ ] Is there a JIRA ticket associated with this PR? Is it referenced
        in the commit message?
   
   - [ ] Does your PR title start with MINIFICPP-XXXX where XXXX is the JIRA number you are trying to resolve? Pay particular attention to the hyphen "-" character.
   
   - [ ] Has your PR been rebased against the latest commit within the target branch (typically main)?
   
   - [ ] Is your initial contribution a single, squashed commit?
   
   ### For code changes:
   - [ ] If adding new dependencies to the code, are these dependencies licensed in a way that is compatible for inclusion under [ASF 2.0](http://www.apache.org/legal/resolved.html#category-a)?
   - [ ] If applicable, have you updated the LICENSE file?
   - [ ] If applicable, have you updated the NOTICE file?
   
   ### For documentation related changes:
   - [ ] Have you ensured that format looks appropriate for the output in which it is rendered?
   
   ### Note:
   Please ensure that once the PR is submitted, you check GitHub Actions CI results for build issues and submit an update to your PR as soon as possible.
   


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

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

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


[GitHub] [nifi-minifi-cpp] lordgamez commented on a diff in pull request #1411: MINIFICPP-1931 Stabilize ResourceQueueTests

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


##########
libminifi/test/unit/ResourceQueueTests.cpp:
##########
@@ -53,23 +55,26 @@ TEST_CASE("maximum_number_of_creatable_resources", "[utils::ResourceQueue]") {
     CHECK(!resources_created.empty());
     CHECK(resources_created.size() <= 2);
   }
+}
 
-  SECTION("No Maximum resources") {
-    LogTestController::getInstance().clear();
-    auto resource_queue = ResourceQueue<int>::create(std::nullopt, 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); }};
+TEST_CASE("Resource limitation is not set to the resource queue", "[utils::ResourceQueue]") {
+  std::shared_ptr<core::logging::Logger> logger_{core::logging::LoggerFactory<ResourceQueue<int>>::getLogger()};
+  LogTestController::getInstance().setTrace<ResourceQueue<int>>();
+  LogTestController::getInstance().clear();
+  auto resource_queue = ResourceQueue<int>::create(std::nullopt, logger_);
+  std::set<int> resources_created;
 
-    thread_one.join();
-    thread_two.join();
-    thread_three.join();
+  auto resource_one = resource_queue->getResource([]{return std::make_unique<int>(1);});
+  auto resource_two = resource_queue->getResource([]{return std::make_unique<int>(2);});
+  auto resource_three = resource_queue->getResource([]{return std::make_unique<int>(3);});
 
-    CHECK(!resources_created.empty());
-    CHECK(!LogTestController::getInstance().contains("Waiting for resource", 0ms));
-    CHECK(LogTestController::getInstance().contains("Number of instances: 3", 0ms));
-    CHECK(resources_created.size() <= 3);
-  }
+  resources_created.emplace(*resource_one);
+  resources_created.emplace(*resource_two);
+  resources_created.emplace(*resource_three);

Review Comment:
   We put 3 different resources in the queue 1, 2 and 3, and if we add those to the set and the set size is 3 then we can be sure that none of the resources was reused, but new resources were created for every instance. I mean you are right that asserting on the `"Number of instances: 3"` is kind of the same, but I would not rely on log messages if not necessary.



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

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

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


[GitHub] [nifi-minifi-cpp] lordgamez commented on a diff in pull request #1411: MINIFICPP-1931 Stabilize ResourceQueueTests

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


##########
libminifi/test/unit/ResourceQueueTests.cpp:
##########
@@ -53,23 +55,26 @@ TEST_CASE("maximum_number_of_creatable_resources", "[utils::ResourceQueue]") {
     CHECK(!resources_created.empty());
     CHECK(resources_created.size() <= 2);
   }
+}
 
-  SECTION("No Maximum resources") {
-    LogTestController::getInstance().clear();
-    auto resource_queue = ResourceQueue<int>::create(std::nullopt, 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); }};
+TEST_CASE("Resource limitation is not set to the resource queue", "[utils::ResourceQueue]") {
+  std::shared_ptr<core::logging::Logger> logger_{core::logging::LoggerFactory<ResourceQueue<int>>::getLogger()};
+  LogTestController::getInstance().setTrace<ResourceQueue<int>>();
+  LogTestController::getInstance().clear();
+  auto resource_queue = ResourceQueue<int>::create(std::nullopt, logger_);
+  std::set<int> resources_created;
 
-    thread_one.join();
-    thread_two.join();
-    thread_three.join();
+  auto resource_one = resource_queue->getResource([]{return std::make_unique<int>(1);});
+  auto resource_two = resource_queue->getResource([]{return std::make_unique<int>(2);});
+  auto resource_three = resource_queue->getResource([]{return std::make_unique<int>(3);});
 
-    CHECK(!resources_created.empty());
-    CHECK(!LogTestController::getInstance().contains("Waiting for resource", 0ms));
-    CHECK(LogTestController::getInstance().contains("Number of instances: 3", 0ms));
-    CHECK(resources_created.size() <= 3);
-  }
+  resources_created.emplace(*resource_one);
+  resources_created.emplace(*resource_two);
+  resources_created.emplace(*resource_three);

Review Comment:
   I'm not sure about your point. The goal of the set is to see if the 3 returned resources are unique and not a previous resource with an already existing value is returned, so at the end the size of the set should be 3. I'm not sure how would that be as easy using just the wrappers.



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

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

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


[GitHub] [nifi-minifi-cpp] fgerlits commented on a diff in pull request #1411: MINIFICPP-1931 Stabilize ResourceQueueTests

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


##########
libminifi/test/unit/ResourceQueueTests.cpp:
##########
@@ -53,23 +55,26 @@ TEST_CASE("maximum_number_of_creatable_resources", "[utils::ResourceQueue]") {
     CHECK(!resources_created.empty());
     CHECK(resources_created.size() <= 2);
   }
+}
 
-  SECTION("No Maximum resources") {
-    LogTestController::getInstance().clear();
-    auto resource_queue = ResourceQueue<int>::create(std::nullopt, 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); }};
+TEST_CASE("Resource limitation is not set to the resource queue", "[utils::ResourceQueue]") {
+  std::shared_ptr<core::logging::Logger> logger_{core::logging::LoggerFactory<ResourceQueue<int>>::getLogger()};
+  LogTestController::getInstance().setTrace<ResourceQueue<int>>();
+  LogTestController::getInstance().clear();
+  auto resource_queue = ResourceQueue<int>::create(std::nullopt, logger_);
+  std::set<int> resources_created;
 
-    thread_one.join();
-    thread_two.join();
-    thread_three.join();
+  auto resource_one = resource_queue->getResource([]{return std::make_unique<int>(1);});
+  auto resource_two = resource_queue->getResource([]{return std::make_unique<int>(2);});
+  auto resource_three = resource_queue->getResource([]{return std::make_unique<int>(3);});
 
-    CHECK(!resources_created.empty());
-    CHECK(!LogTestController::getInstance().contains("Waiting for resource", 0ms));
-    CHECK(LogTestController::getInstance().contains("Number of instances: 3", 0ms));
-    CHECK(resources_created.size() <= 3);
-  }
+  resources_created.emplace(*resource_one);
+  resources_created.emplace(*resource_two);
+  resources_created.emplace(*resource_three);

Review Comment:
   Maybe I'm missing something, but I don't see why we need to copy the resources into the set.  Would this not do the same?
   ```c++
   TEST_CASE("Resource limitation is not set to the resource queue", "[utils::ResourceQueue]") {
     std::shared_ptr<core::logging::Logger> logger_{core::logging::LoggerFactory<ResourceQueue<int>>::getLogger()};
     LogTestController::getInstance().setTrace<ResourceQueue<int>>();
     LogTestController::getInstance().clear();
   
     auto resource_queue = ResourceQueue<int>::create(std::nullopt, logger_);
   
     auto resource_wrapper_one = resource_queue->getResource([]{return std::make_unique<int>(1);});
     auto resource_wrapper_two = resource_queue->getResource([]{return std::make_unique<int>(2);});
     auto resource_wrapper_three = resource_queue->getResource([]{return std::make_unique<int>(3);});
   
     CHECK(!LogTestController::getInstance().contains("Waiting for resource", 0ms));
     CHECK(LogTestController::getInstance().contains("Number of instances: 3", 0ms));
   }
   ```



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

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

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


[GitHub] [nifi-minifi-cpp] lordgamez commented on pull request #1411: MINIFICPP-1931 Stabilize ResourceQueueTests

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

   > Thanks! I asked @martinzink to do this, but fixing it instead of pestering someone else to fix it is also an option. :-)
   > 
   > Also, @martinzink found another, rarer, race condition in the "No Maximum resources" section of the test, please talk to him about it.
   
   I was just there stabilizng tests when I saw this so I just took it :)
   
   I talked to @martinzink and fixed the concurrency issue in 92dcdb522e440cd6aa910286f4f36a7ccd4aadaa


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

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

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


[GitHub] [nifi-minifi-cpp] fgerlits commented on a diff in pull request #1411: MINIFICPP-1931 Stabilize ResourceQueueTests

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


##########
libminifi/test/unit/ResourceQueueTests.cpp:
##########
@@ -53,23 +55,26 @@ TEST_CASE("maximum_number_of_creatable_resources", "[utils::ResourceQueue]") {
     CHECK(!resources_created.empty());
     CHECK(resources_created.size() <= 2);
   }
+}
 
-  SECTION("No Maximum resources") {
-    LogTestController::getInstance().clear();
-    auto resource_queue = ResourceQueue<int>::create(std::nullopt, 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); }};
+TEST_CASE("Resource limitation is not set to the resource queue", "[utils::ResourceQueue]") {
+  std::shared_ptr<core::logging::Logger> logger_{core::logging::LoggerFactory<ResourceQueue<int>>::getLogger()};
+  LogTestController::getInstance().setTrace<ResourceQueue<int>>();
+  LogTestController::getInstance().clear();
+  auto resource_queue = ResourceQueue<int>::create(std::nullopt, logger_);
+  std::set<int> resources_created;
 
-    thread_one.join();
-    thread_two.join();
-    thread_three.join();
+  auto resource_one = resource_queue->getResource([]{return std::make_unique<int>(1);});
+  auto resource_two = resource_queue->getResource([]{return std::make_unique<int>(2);});
+  auto resource_three = resource_queue->getResource([]{return std::make_unique<int>(3);});
 
-    CHECK(!resources_created.empty());
-    CHECK(!LogTestController::getInstance().contains("Waiting for resource", 0ms));
-    CHECK(LogTestController::getInstance().contains("Number of instances: 3", 0ms));
-    CHECK(resources_created.size() <= 3);
-  }
+  resources_created.emplace(*resource_one);
+  resources_created.emplace(*resource_two);
+  resources_created.emplace(*resource_three);

Review Comment:
   I don't think we need the `resources_created` set, as the `resource_*` variables hold onto the wrappers already.
   
   Also, maybe rename `resource_*` to `resource_wrapper_*`?



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

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

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


[GitHub] [nifi-minifi-cpp] fgerlits commented on a diff in pull request #1411: MINIFICPP-1931 Stabilize ResourceQueueTests

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


##########
libminifi/test/unit/ResourceQueueTests.cpp:
##########
@@ -53,23 +55,26 @@ TEST_CASE("maximum_number_of_creatable_resources", "[utils::ResourceQueue]") {
     CHECK(!resources_created.empty());
     CHECK(resources_created.size() <= 2);
   }
+}
 
-  SECTION("No Maximum resources") {
-    LogTestController::getInstance().clear();
-    auto resource_queue = ResourceQueue<int>::create(std::nullopt, 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); }};
+TEST_CASE("Resource limitation is not set to the resource queue", "[utils::ResourceQueue]") {
+  std::shared_ptr<core::logging::Logger> logger_{core::logging::LoggerFactory<ResourceQueue<int>>::getLogger()};
+  LogTestController::getInstance().setTrace<ResourceQueue<int>>();
+  LogTestController::getInstance().clear();
+  auto resource_queue = ResourceQueue<int>::create(std::nullopt, logger_);
+  std::set<int> resources_created;
 
-    thread_one.join();
-    thread_two.join();
-    thread_three.join();
+  auto resource_one = resource_queue->getResource([]{return std::make_unique<int>(1);});
+  auto resource_two = resource_queue->getResource([]{return std::make_unique<int>(2);});
+  auto resource_three = resource_queue->getResource([]{return std::make_unique<int>(3);});
 
-    CHECK(!resources_created.empty());
-    CHECK(!LogTestController::getInstance().contains("Waiting for resource", 0ms));
-    CHECK(LogTestController::getInstance().contains("Number of instances: 3", 0ms));
-    CHECK(resources_created.size() <= 3);
-  }
+  resources_created.emplace(*resource_one);
+  resources_created.emplace(*resource_two);
+  resources_created.emplace(*resource_three);

Review Comment:
   OK, I see.  To me, the shorter version would be clearer, but I don't mind keeping the extra check.



-- 
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 #1411: MINIFICPP-1931 Stabilize ResourceQueueTests

Posted by GitBox <gi...@apache.org>.
fgerlits closed pull request #1411: MINIFICPP-1931 Stabilize ResourceQueueTests
URL: https://github.com/apache/nifi-minifi-cpp/pull/1411


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

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

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


[GitHub] [nifi-minifi-cpp] lordgamez commented on a diff in pull request #1411: MINIFICPP-1931 Stabilize ResourceQueueTests

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


##########
libminifi/test/unit/ResourceQueueTests.cpp:
##########
@@ -28,19 +28,21 @@ using namespace std::literals::chrono_literals;
 
 namespace org::apache::nifi::minifi::utils::testing {
 
-TEST_CASE("maximum_number_of_creatable_resources", "[utils::ResourceQueue]") {
+TEST_CASE("Limiting resource queue to a maximum of 2 resources", "[utils::ResourceQueue]") {
   std::shared_ptr<core::logging::Logger> logger_{core::logging::LoggerFactory<ResourceQueue<int>>::getLogger()};
   LogTestController::getInstance().setTrace<ResourceQueue<int>>();
 
+  std::mutex resources_created_mutex;
   std::set<int> resources_created;
 
   auto worker = [&](int value, const std::shared_ptr<ResourceQueue<int>>& resource_queue) {
     auto resource = resource_queue->getResource([value]{return std::make_unique<int>(value);});
     std::this_thread::sleep_for(10ms);
+    std::lock_guard<std::mutex> lock(resources_created_mutex);
     resources_created.emplace(*resource);
   };
 
-  SECTION("Maximum 2 resources") {
+  SECTION("") {

Review Comment:
   Updated in 5f0605b2e6a9017c1768f37968c050b9e551f812



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

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

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


[GitHub] [nifi-minifi-cpp] fgerlits commented on a diff in pull request #1411: MINIFICPP-1931 Stabilize ResourceQueueTests

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


##########
libminifi/test/unit/ResourceQueueTests.cpp:
##########
@@ -53,23 +55,26 @@ TEST_CASE("maximum_number_of_creatable_resources", "[utils::ResourceQueue]") {
     CHECK(!resources_created.empty());
     CHECK(resources_created.size() <= 2);
   }
+}
 
-  SECTION("No Maximum resources") {
-    LogTestController::getInstance().clear();
-    auto resource_queue = ResourceQueue<int>::create(std::nullopt, 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); }};
+TEST_CASE("Resource limitation is not set to the resource queue", "[utils::ResourceQueue]") {
+  std::shared_ptr<core::logging::Logger> logger_{core::logging::LoggerFactory<ResourceQueue<int>>::getLogger()};
+  LogTestController::getInstance().setTrace<ResourceQueue<int>>();
+  LogTestController::getInstance().clear();
+  auto resource_queue = ResourceQueue<int>::create(std::nullopt, logger_);
+  std::set<int> resources_created;
 
-    thread_one.join();
-    thread_two.join();
-    thread_three.join();
+  auto resource_one = resource_queue->getResource([]{return std::make_unique<int>(1);});
+  auto resource_two = resource_queue->getResource([]{return std::make_unique<int>(2);});
+  auto resource_three = resource_queue->getResource([]{return std::make_unique<int>(3);});
 
-    CHECK(!resources_created.empty());
-    CHECK(!LogTestController::getInstance().contains("Waiting for resource", 0ms));
-    CHECK(LogTestController::getInstance().contains("Number of instances: 3", 0ms));
-    CHECK(resources_created.size() <= 3);
-  }
+  resources_created.emplace(*resource_one);
+  resources_created.emplace(*resource_two);
+  resources_created.emplace(*resource_three);

Review Comment:
   Maybe I'm missing something, but I don't see why we need to copy the resources into the set.  Would this not do the same?
   ```c++
   TEST_CASE("Resource limitation is not set to the resource queue", "[utils::ResourceQueue]") {
     std::shared_ptr<core::logging::Logger> logger_{core::logging::LoggerFactory<ResourceQueue<int>>::getLogger()};
     LogTestController::getInstance().setTrace<ResourceQueue<int>>();
     LogTestController::getInstance().clear();
   
     auto resource_queue = ResourceQueue<int>::create(std::nullopt, logger_);
   
     auto resource_wrapper_one = resource_queue->getResource([]{return std::make_unique<int>(1);});
     auto resource_wrapper_two = resource_queue->getResource([]{return std::make_unique<int>(2);});
     auto resource_wrapper_three = resource_queue->getResource([]{return std::make_unique<int>(3);});
   
     CHECK(!LogTestController::getInstance().contains("Waiting for resource", 0ms));
     CHECK(LogTestController::getInstance().contains("Number of instances: 3", 0ms));
     CHECK(resources_created.size() == 3);
   }
   ```



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

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

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


[GitHub] [nifi-minifi-cpp] fgerlits commented on a diff in pull request #1411: MINIFICPP-1931 Stabilize ResourceQueueTests

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


##########
libminifi/test/unit/ResourceQueueTests.cpp:
##########
@@ -28,19 +28,21 @@ using namespace std::literals::chrono_literals;
 
 namespace org::apache::nifi::minifi::utils::testing {
 
-TEST_CASE("maximum_number_of_creatable_resources", "[utils::ResourceQueue]") {
+TEST_CASE("Limiting resource queue to a maximum of 2 resources", "[utils::ResourceQueue]") {
   std::shared_ptr<core::logging::Logger> logger_{core::logging::LoggerFactory<ResourceQueue<int>>::getLogger()};
   LogTestController::getInstance().setTrace<ResourceQueue<int>>();
 
+  std::mutex resources_created_mutex;
   std::set<int> resources_created;
 
   auto worker = [&](int value, const std::shared_ptr<ResourceQueue<int>>& resource_queue) {
     auto resource = resource_queue->getResource([value]{return std::make_unique<int>(value);});
     std::this_thread::sleep_for(10ms);
+    std::lock_guard<std::mutex> lock(resources_created_mutex);
     resources_created.emplace(*resource);
   };
 
-  SECTION("Maximum 2 resources") {
+  SECTION("") {

Review Comment:
   there is no point of having a SECTION if there is only one, I would remove it



##########
libminifi/test/unit/ResourceQueueTests.cpp:
##########
@@ -53,23 +55,26 @@ TEST_CASE("maximum_number_of_creatable_resources", "[utils::ResourceQueue]") {
     CHECK(!resources_created.empty());
     CHECK(resources_created.size() <= 2);
   }
+}
 
-  SECTION("No Maximum resources") {
-    LogTestController::getInstance().clear();
-    auto resource_queue = ResourceQueue<int>::create(std::nullopt, 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); }};
+TEST_CASE("Resource limitation is not set to the resource queue", "[utils::ResourceQueue]") {
+  std::shared_ptr<core::logging::Logger> logger_{core::logging::LoggerFactory<ResourceQueue<int>>::getLogger()};
+  LogTestController::getInstance().setTrace<ResourceQueue<int>>();
+  LogTestController::getInstance().clear();
+  auto resource_queue = ResourceQueue<int>::create(std::nullopt, logger_);
+  std::set<int> resources_created;
 
-    thread_one.join();
-    thread_two.join();
-    thread_three.join();
+  auto resource_one = resource_queue->getResource([]{return std::make_unique<int>(1);});
+  auto resource_two = resource_queue->getResource([]{return std::make_unique<int>(2);});
+  auto resource_three = resource_queue->getResource([]{return std::make_unique<int>(3);});
 
-    CHECK(!resources_created.empty());
-    CHECK(!LogTestController::getInstance().contains("Waiting for resource", 0ms));
-    CHECK(LogTestController::getInstance().contains("Number of instances: 3", 0ms));
-    CHECK(resources_created.size() <= 3);
-  }
+  resources_created.emplace(*resource_one);
+  resources_created.emplace(*resource_two);
+  resources_created.emplace(*resource_three);

Review Comment:
   I don't think we need the `resources_created` set, as the `resource_x` variables hold onto the wrappers already.



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