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/10/17 12:41:04 UTC

[GitHub] [nifi-minifi-cpp] adamdebreceni opened a new pull request, #1435: MINIFICPP-1959 - Avoid dangling content references when using VolatileFlowFileRepository

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

   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] adamdebreceni commented on a diff in pull request #1435: MINIFICPP-1959 - Avoid dangling content references when using VolatileFlowFileRepository

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


##########
libminifi/test/unit/ProcessSessionTests.cpp:
##########
@@ -126,3 +132,44 @@ TEST_CASE("ProcessSession::read can read zero length flowfiles without crash", "
   ContentRepositoryDependentTests::testReadFromZeroLengthFlowFile(std::make_shared<core::repository::VolatileContentRepository>());
   ContentRepositoryDependentTests::testReadFromZeroLengthFlowFile(std::make_shared<core::repository::FileSystemRepository>());
 }
+
+struct VolatileFlowFileRepositoryTestAccessor {
+  METHOD_ACCESSOR(flush);
+};
+
+class TestVolatileFlowFileRepository : public core::repository::VolatileFlowFileRepository {
+ public:
+  explicit TestVolatileFlowFileRepository(std::string name) : core::SerializableComponent(std::move(name)) {}

Review Comment:
   `SerializableComponent` is a virtual base class without default initializer, so we have to directly initialize it in every derived class



-- 
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 #1435: MINIFICPP-1959 - Avoid dangling content references when using VolatileFlowFileRepository

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


##########
libminifi/test/unit/ProcessSessionTests.cpp:
##########
@@ -126,3 +132,44 @@ TEST_CASE("ProcessSession::read can read zero length flowfiles without crash", "
   ContentRepositoryDependentTests::testReadFromZeroLengthFlowFile(std::make_shared<core::repository::VolatileContentRepository>());
   ContentRepositoryDependentTests::testReadFromZeroLengthFlowFile(std::make_shared<core::repository::FileSystemRepository>());
 }
+
+struct VolatileFlowFileRepositoryTestAccessor {
+  METHOD_ACCESSOR(flush);
+};
+
+class TestVolatileFlowFileRepository : public core::repository::VolatileFlowFileRepository {
+ public:
+  explicit TestVolatileFlowFileRepository(std::string name) : core::SerializableComponent(std::move(name)) {}

Review Comment:
   why is this not
   ```suggestion
     explicit TestVolatileFlowFileRepository(std::string name) : core::repository::VolatileFlowFileRepository(std::move(name)) {}
   ```
   ?



##########
libminifi/include/core/repository/VolatileRepository.h:
##########
@@ -256,7 +257,7 @@ bool VolatileRepository<KeyType, RepositoryType>::Put(const KeyType& key, const
   } while (!updated);
   current_size_ += size;
 
-  logger_->log_debug("VolatileRepository -- put %u %u", current_size_.load(), current_index_.load());
+  logger_->log_debug("VolatileRepository -- put %zu " PRIu32, current_size_.load(), current_index_.load());

Review Comment:
   there is a `%` missing before `PRIu32`



-- 
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] adamdebreceni commented on a diff in pull request #1435: MINIFICPP-1959 - Avoid dangling content references when using VolatileFlowFileRepository

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


##########
libminifi/include/core/repository/VolatileRepository.h:
##########
@@ -256,7 +257,7 @@ bool VolatileRepository<KeyType, RepositoryType>::Put(const KeyType& key, const
   } while (!updated);
   current_size_ += size;
 
-  logger_->log_debug("VolatileRepository -- put %u %u", current_size_.load(), current_index_.load());
+  logger_->log_debug("VolatileRepository -- put %zu " PRIu32, current_size_.load(), current_index_.load());

Review Comment:
   added



-- 
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] adam-markovics commented on a diff in pull request #1435: MINIFICPP-1959 - Avoid dangling content references when using VolatileFlowFileRepository

Posted by GitBox <gi...@apache.org>.
adam-markovics commented on code in PR #1435:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1435#discussion_r1013850774


##########
libminifi/test/unit/ProcessSessionTests.cpp:
##########
@@ -138,3 +144,44 @@ TEST_CASE("ProcessSession::read can read zero length flowfiles without crash", "
   ContentRepositoryDependentTests::testReadFromZeroLengthFlowFile(std::make_shared<core::repository::VolatileContentRepository>());
   ContentRepositoryDependentTests::testReadFromZeroLengthFlowFile(std::make_shared<core::repository::FileSystemRepository>());
 }
+
+struct VolatileFlowFileRepositoryTestAccessor {
+  METHOD_ACCESSOR(flush);
+};
+
+class TestVolatileFlowFileRepository : public core::repository::VolatileFlowFileRepository {
+ public:
+  explicit TestVolatileFlowFileRepository(const std::string& name) : core::SerializableComponent(name) {}
+
+  bool MultiPut(const std::vector<std::pair<std::string, std::unique_ptr<minifi::io::BufferStream>>>& data) override {
+    auto flush_on_exit = gsl::finally([&] {VolatileFlowFileRepositoryTestAccessor::call_flush(*this);});
+    return VolatileFlowFileRepository::MultiPut(data);
+  }
+};
+
+TEST_CASE("ProcessSession::commit avoids dangling ResourceClaims when using VolatileFlowFileRepository", "[incrementbefore]") {
+  auto configuration = std::make_shared<minifi::Configure>();
+  configuration->set(minifi::Configure::nifi_volatile_repository_options_flowfile_max_count, "2");
+  auto ff_repo = std::make_shared<TestVolatileFlowFileRepository>("flowfile");
+  Fixture fixture({
+    .configuration = std::move(configuration),
+    .flow_file_repo = ff_repo
+  });
+  auto& session = fixture.processSession();
+
+  const auto flow_file_1 = session.create();
+  const auto flow_file_2 = session.create();
+  const auto flow_file_3 = session.create();
+  session.transfer(flow_file_1, Success);
+  session.transfer(flow_file_2, Success);
+  session.transfer(flow_file_3, Success);
+  session.commit();
+
+  REQUIRE(flow_file_1->getResourceClaim()->getFlowFileRecordOwnedCount() == 1);
+  REQUIRE(flow_file_2->getResourceClaim()->getFlowFileRecordOwnedCount() == 2);
+  REQUIRE(flow_file_3->getResourceClaim()->getFlowFileRecordOwnedCount() == 2);

Review Comment:
   Thanks, it's more clear now!



-- 
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] adamdebreceni commented on a diff in pull request #1435: MINIFICPP-1959 - Avoid dangling content references when using VolatileFlowFileRepository

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


##########
libminifi/test/TestBase.h:
##########
@@ -331,10 +331,19 @@ class TestPlan {
 
 class TestController {
  public:
+  struct PlanConfig {
+    std::shared_ptr<minifi::Configure> configuration = {};
+    const char* state_dir = nullptr;

Review Comment:
   changed it to `std::optional<std::filesystem::path>`



-- 
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 #1435: MINIFICPP-1959 - Avoid dangling content references when using VolatileFlowFileRepository

Posted by GitBox <gi...@apache.org>.
fgerlits closed pull request #1435: MINIFICPP-1959 - Avoid dangling content references when using VolatileFlowFileRepository
URL: https://github.com/apache/nifi-minifi-cpp/pull/1435


-- 
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] adamdebreceni commented on a diff in pull request #1435: MINIFICPP-1959 - Avoid dangling content references when using VolatileFlowFileRepository

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


##########
libminifi/test/unit/ProcessSessionTests.cpp:
##########
@@ -138,3 +144,44 @@ TEST_CASE("ProcessSession::read can read zero length flowfiles without crash", "
   ContentRepositoryDependentTests::testReadFromZeroLengthFlowFile(std::make_shared<core::repository::VolatileContentRepository>());
   ContentRepositoryDependentTests::testReadFromZeroLengthFlowFile(std::make_shared<core::repository::FileSystemRepository>());
 }
+
+struct VolatileFlowFileRepositoryTestAccessor {
+  METHOD_ACCESSOR(flush);
+};
+
+class TestVolatileFlowFileRepository : public core::repository::VolatileFlowFileRepository {
+ public:
+  explicit TestVolatileFlowFileRepository(const std::string& name) : core::SerializableComponent(name) {}
+
+  bool MultiPut(const std::vector<std::pair<std::string, std::unique_ptr<minifi::io::BufferStream>>>& data) override {
+    auto flush_on_exit = gsl::finally([&] {VolatileFlowFileRepositoryTestAccessor::call_flush(*this);});
+    return VolatileFlowFileRepository::MultiPut(data);
+  }
+};
+
+TEST_CASE("ProcessSession::commit avoids dangling ResourceClaims when using VolatileFlowFileRepository", "[incrementbefore]") {
+  auto configuration = std::make_shared<minifi::Configure>();
+  configuration->set(minifi::Configure::nifi_volatile_repository_options_flowfile_max_count, "2");
+  auto ff_repo = std::make_shared<TestVolatileFlowFileRepository>("flowfile");
+  Fixture fixture({
+    .configuration = std::move(configuration),
+    .flow_file_repo = ff_repo
+  });
+  auto& session = fixture.processSession();
+
+  const auto flow_file_1 = session.create();
+  const auto flow_file_2 = session.create();
+  const auto flow_file_3 = session.create();
+  session.transfer(flow_file_1, Success);
+  session.transfer(flow_file_2, Success);
+  session.transfer(flow_file_3, Success);
+  session.commit();
+
+  REQUIRE(flow_file_1->getResourceClaim()->getFlowFileRecordOwnedCount() == 1);
+  REQUIRE(flow_file_2->getResourceClaim()->getFlowFileRecordOwnedCount() == 2);
+  REQUIRE(flow_file_3->getResourceClaim()->getFlowFileRecordOwnedCount() == 2);

Review Comment:
   added comment, the shared_ptr on the stack is the other owner



-- 
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] adam-markovics commented on a diff in pull request #1435: MINIFICPP-1959 - Avoid dangling content references when using VolatileFlowFileRepository

Posted by GitBox <gi...@apache.org>.
adam-markovics commented on code in PR #1435:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1435#discussion_r1011863921


##########
libminifi/src/core/ProcessSession.cpp:
##########
@@ -958,54 +958,72 @@ void ProcessSession::persistFlowFilesBeforeTransfer(
   auto flowFileRepo = process_context_->getFlowFileRepository();
   auto contentRepo = process_context_->getContentRepository();
 
-  for (auto& [target, flows] : transactionMap) {
-    const auto connection = dynamic_cast<Connection*>(target);
-    const bool shouldDropEmptyFiles = connection && connection->getDropEmptyFlowFiles();
-    for (auto &ff : flows) {
-      if (shouldDropEmptyFiles && ff->getSize() == 0) {
-        // the receiver will drop this FF
-        continue;
+  enum class Type {
+    Dropped, Transferred
+  };
+
+  auto forEachFlowFile = [&] (Type type, auto fn) {
+    for (auto& [target, flows] : transactionMap) {
+      const auto connection = dynamic_cast<Connection*>(target);
+      const bool shouldDropEmptyFiles = connection && connection->getDropEmptyFlowFiles();
+      for (auto &ff : flows) {
+        auto snapshotIt = modifiedFlowFiles.find(ff->getUUID());
+        auto original = snapshotIt != modifiedFlowFiles.end() ? snapshotIt->second.snapshot : nullptr;
+        if (shouldDropEmptyFiles && ff->getSize() == 0) {
+          // the receiver will drop this FF
+          if (type == Type::Dropped) {
+            fn(ff, original);
+          }
+        } else {
+          if (type == Type::Transferred) {
+            fn(ff, original);
+          }
+        }
       }
+    }
+  };
 
-      std::unique_ptr<io::BufferStream> stream(new io::BufferStream());
-      std::static_pointer_cast<FlowFileRecord>(ff)->Serialize(*stream);
+  // collect serialized flowfiles
+  forEachFlowFile(Type::Transferred, [&] (auto& ff, auto& /*original*/) {
+    std::unique_ptr<io::BufferStream> stream(new io::BufferStream());

Review Comment:
   make_unique is more modern



##########
libminifi/test/TestBase.h:
##########
@@ -331,10 +331,19 @@ class TestPlan {
 
 class TestController {
  public:
+  struct PlanConfig {
+    std::shared_ptr<minifi::Configure> configuration = {};
+    const char* state_dir = nullptr;

Review Comment:
   I think `string_view` would be better, as pointers don't store either. Default member initializer would no longer be necessary, neither for other members (shared_ptrs).



##########
libminifi/test/unit/ProcessSessionTests.cpp:
##########
@@ -138,3 +144,44 @@ TEST_CASE("ProcessSession::read can read zero length flowfiles without crash", "
   ContentRepositoryDependentTests::testReadFromZeroLengthFlowFile(std::make_shared<core::repository::VolatileContentRepository>());
   ContentRepositoryDependentTests::testReadFromZeroLengthFlowFile(std::make_shared<core::repository::FileSystemRepository>());
 }
+
+struct VolatileFlowFileRepositoryTestAccessor {
+  METHOD_ACCESSOR(flush);
+};
+
+class TestVolatileFlowFileRepository : public core::repository::VolatileFlowFileRepository {
+ public:
+  explicit TestVolatileFlowFileRepository(const std::string& name) : core::SerializableComponent(name) {}
+
+  bool MultiPut(const std::vector<std::pair<std::string, std::unique_ptr<minifi::io::BufferStream>>>& data) override {
+    auto flush_on_exit = gsl::finally([&] {VolatileFlowFileRepositoryTestAccessor::call_flush(*this);});
+    return VolatileFlowFileRepository::MultiPut(data);
+  }
+};
+
+TEST_CASE("ProcessSession::commit avoids dangling ResourceClaims when using VolatileFlowFileRepository", "[incrementbefore]") {
+  auto configuration = std::make_shared<minifi::Configure>();
+  configuration->set(minifi::Configure::nifi_volatile_repository_options_flowfile_max_count, "2");
+  auto ff_repo = std::make_shared<TestVolatileFlowFileRepository>("flowfile");
+  Fixture fixture({
+    .configuration = std::move(configuration),
+    .flow_file_repo = ff_repo
+  });
+  auto& session = fixture.processSession();
+
+  const auto flow_file_1 = session.create();
+  const auto flow_file_2 = session.create();
+  const auto flow_file_3 = session.create();
+  session.transfer(flow_file_1, Success);
+  session.transfer(flow_file_2, Success);
+  session.transfer(flow_file_3, Success);
+  session.commit();
+
+  REQUIRE(flow_file_1->getResourceClaim()->getFlowFileRecordOwnedCount() == 1);
+  REQUIRE(flow_file_2->getResourceClaim()->getFlowFileRecordOwnedCount() == 2);
+  REQUIRE(flow_file_3->getResourceClaim()->getFlowFileRecordOwnedCount() == 2);

Review Comment:
   Why is it 1, 2, 2? I thought it should be 0, 1, 1. Who is the other owner than `ff_repo`?



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