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/03/17 08:53:24 UTC

[GitHub] [nifi-minifi-cpp] adam-markovics opened a new pull request #1284: MINIFICPP-1776 - Parallel heartbeat and flow update can cause crash

adam-markovics opened a new pull request #1284:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1284


   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] adam-markovics commented on a change in pull request #1284: MINIFICPP-1776 - Parallel heartbeat and flow update can cause crash

Posted by GitBox <gi...@apache.org>.
adam-markovics commented on a change in pull request #1284:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1284#discussion_r832339939



##########
File path: libminifi/src/FlowController.cpp
##########
@@ -438,30 +432,43 @@ std::shared_ptr<state::response::ResponseNode> FlowController::getAgentManifest(
   return agentInfo;
 }
 
+void FlowController::executeOnAllComponents(std::function<void(state::StateController*)> func) {
+  std::lock_guard<std::recursive_mutex> lock(mutex_);
+  for (auto* component: getAllComponents()) {
+    func(component);
+  }

Review comment:
       Done.

##########
File path: libminifi/src/FlowController.cpp
##########
@@ -506,31 +513,36 @@ std::map<std::string, std::unique_ptr<io::InputStream>> FlowController::getDebug
   return debug_info;
 }
 
-void FlowController::getAllProcessorControllers(std::vector<state::StateController*>& controllerVec,
-                                              const std::function<std::unique_ptr<state::ProcessorController>(core::Processor&)>& controllerFactory) {
-  std::lock_guard<std::recursive_mutex> lock(mutex_);
-
+std::vector<state::StateController*> FlowController::getAllProcessorControllers(
+        const std::function<std::unique_ptr<state::ProcessorController>(core::Processor&)>& controllerFactory) {
+  std::vector<state::StateController*> controllerVec{this};
   std::vector<core::Processor*> processorVec;
   root_->getAllProcessors(processorVec);
 
   for (const auto& processor : processorVec) {
-    // find controller for processor, if it doesn't exist, create one
+    // reference to the existing or newly created controller
     auto& controller = processor_to_controller_[processor->getUUID()];
     if (!controller) {
       controller = controllerFactory(*processor);
     }
     controllerVec.push_back(controller.get());
   }
+
+  return controllerVec;
 }
 
-void FlowController::getProcessorController(const std::string& name, std::vector<state::StateController*>& controllerVec,
-                                          const std::function<std::unique_ptr<state::ProcessorController>(core::Processor&)>& controllerFactory) {
+state::StateController* FlowController::getProcessorController(const std::string& name, const std::function<std::unique_ptr<state::ProcessorController>(core::Processor&)>& controllerFactory) {
   auto* processor = root_->findProcessorByName(name);
-  auto& controller = processor_to_controller_[processor->getUUID()];
-  if (!controller) {
-    controller = controllerFactory(*processor);
+  if (processor == nullptr) {
+    logger_->log_error("Could not get processor controller for requested name \"%s\", because processor was not found either", name);
+  }

Review comment:
       Done.




-- 
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 change in pull request #1284: MINIFICPP-1776 - Parallel heartbeat and flow update can cause crash

Posted by GitBox <gi...@apache.org>.
adam-markovics commented on a change in pull request #1284:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1284#discussion_r832046869



##########
File path: libminifi/src/FlowController.cpp
##########
@@ -523,14 +537,14 @@ void FlowController::getAllProcessorControllers(std::vector<state::StateControll
   }
 }
 
-void FlowController::getProcessorController(const std::string& name, std::vector<state::StateController*>& controllerVec,
+void FlowController::getProcessorController(const std::string& name, state::StateController*& controller,
                                           const std::function<std::unique_ptr<state::ProcessorController>(core::Processor&)>& controllerFactory) {
   auto* processor = root_->findProcessorByName(name);
-  auto& controller = processor_to_controller_[processor->getUUID()];
-  if (!controller) {
-    controller = controllerFactory(*processor);
+  auto& foundController = processor_to_controller_[processor->getUUID()];

Review comment:
       Done.




-- 
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 change in pull request #1284: MINIFICPP-1776 - Parallel heartbeat and flow update can cause crash

Posted by GitBox <gi...@apache.org>.
fgerlits commented on a change in pull request #1284:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1284#discussion_r832085417



##########
File path: libminifi/src/FlowController.cpp
##########
@@ -506,31 +513,36 @@ std::map<std::string, std::unique_ptr<io::InputStream>> FlowController::getDebug
   return debug_info;
 }
 
-void FlowController::getAllProcessorControllers(std::vector<state::StateController*>& controllerVec,
-                                              const std::function<std::unique_ptr<state::ProcessorController>(core::Processor&)>& controllerFactory) {
-  std::lock_guard<std::recursive_mutex> lock(mutex_);
-
+std::vector<state::StateController*> FlowController::getAllProcessorControllers(
+        const std::function<std::unique_ptr<state::ProcessorController>(core::Processor&)>& controllerFactory) {
+  std::vector<state::StateController*> controllerVec{this};
   std::vector<core::Processor*> processorVec;
   root_->getAllProcessors(processorVec);
 
   for (const auto& processor : processorVec) {
-    // find controller for processor, if it doesn't exist, create one
+    // reference to the existing or newly created controller
     auto& controller = processor_to_controller_[processor->getUUID()];
     if (!controller) {
       controller = controllerFactory(*processor);
     }
     controllerVec.push_back(controller.get());
   }
+
+  return controllerVec;
 }
 
-void FlowController::getProcessorController(const std::string& name, std::vector<state::StateController*>& controllerVec,
-                                          const std::function<std::unique_ptr<state::ProcessorController>(core::Processor&)>& controllerFactory) {
+state::StateController* FlowController::getProcessorController(const std::string& name, const std::function<std::unique_ptr<state::ProcessorController>(core::Processor&)>& controllerFactory) {
   auto* processor = root_->findProcessorByName(name);
-  auto& controller = processor_to_controller_[processor->getUUID()];
-  if (!controller) {
-    controller = controllerFactory(*processor);
+  if (processor == nullptr) {
+    logger_->log_error("Could not get processor controller for requested name \"%s\", because processor was not found either", name);
+  }

Review comment:
       logging before crashing is an improvement, but not crashing is even better:
   ```suggestion
     if (processor == nullptr) {
       logger_->log_error("Could not get processor controller for requested name \"%s\", because processor was not found either", name);
       return nullptr;
     }
   ```




-- 
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 change in pull request #1284: MINIFICPP-1776 - Parallel heartbeat and flow update can cause crash

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #1284:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1284#discussion_r832298512



##########
File path: libminifi/src/FlowController.cpp
##########
@@ -438,30 +432,43 @@ std::shared_ptr<state::response::ResponseNode> FlowController::getAgentManifest(
   return agentInfo;
 }
 
+void FlowController::executeOnAllComponents(std::function<void(state::StateController*)> func) {
+  std::lock_guard<std::recursive_mutex> lock(mutex_);
+  for (auto* component: getAllComponents()) {
+    func(component);
+  }
+}
+
+void FlowController::executeOnComponent(const std::string &name, std::function<void(state::StateController*)> func) {
+  std::lock_guard<std::recursive_mutex> lock(mutex_);
+  if (auto* component = getComponent(name); component != nullptr) {
+    func(component);

Review comment:
       thanks




-- 
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 change in pull request #1284: MINIFICPP-1776 - Parallel heartbeat and flow update can cause crash

Posted by GitBox <gi...@apache.org>.
adam-markovics commented on a change in pull request #1284:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1284#discussion_r832042678



##########
File path: libminifi/src/FlowController.cpp
##########
@@ -523,14 +537,14 @@ void FlowController::getAllProcessorControllers(std::vector<state::StateControll
   }
 }
 
-void FlowController::getProcessorController(const std::string& name, std::vector<state::StateController*>& controllerVec,
+void FlowController::getProcessorController(const std::string& name, state::StateController*& controller,
                                           const std::function<std::unique_ptr<state::ProcessorController>(core::Processor&)>& controllerFactory) {
   auto* processor = root_->findProcessorByName(name);
-  auto& controller = processor_to_controller_[processor->getUUID()];
-  if (!controller) {
-    controller = controllerFactory(*processor);
+  auto& foundController = processor_to_controller_[processor->getUUID()];
+  if (!foundController) {
+    foundController = controllerFactory(*processor);
   }
-  controllerVec.push_back(controller.get());
+  controller = foundController.get();

Review comment:
       It's not dangling, because `foundController` is a reference to the found value in the map. The created controller is going to be stored there.




-- 
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 change in pull request #1284: MINIFICPP-1776 - Parallel heartbeat and flow update can cause crash

Posted by GitBox <gi...@apache.org>.
adam-markovics commented on a change in pull request #1284:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1284#discussion_r831131778



##########
File path: extensions/http-curl/tests/C2ClearCoreComponentStateTest.cpp
##########
@@ -51,10 +51,21 @@ class VerifyC2ClearCoreComponentState : public VerifyC2Base {
 
  protected:
   void updateProperties(minifi::FlowController& flow_controller) override {
-    dynamic_cast<minifi::state::ProcessorController*>(flow_controller.getComponents("TailFile1")[0])
-        ->getProcessor()->setProperty(minifi::processors::TailFile::FileName, test_file_1_);
-    dynamic_cast<minifi::state::ProcessorController*>(flow_controller.getComponents("TailFile2")[0])
-        ->getProcessor()->setProperty(minifi::processors::TailFile::FileName, test_file_2_);
+    size_t componentIdx = 0;
+    auto setFileName = [&componentIdx] (const std::string& fileName, minifi::state::StateController* component){

Review comment:
       Not present anymore.




-- 
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 change in pull request #1284: MINIFICPP-1776 - Parallel heartbeat and flow update can cause crash

Posted by GitBox <gi...@apache.org>.
fgerlits commented on a change in pull request #1284:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1284#discussion_r832014350



##########
File path: libminifi/src/FlowController.cpp
##########
@@ -438,6 +440,20 @@ std::shared_ptr<state::response::ResponseNode> FlowController::getAgentManifest(
   return agentInfo;
 }
 
+void FlowController::executeOnAllComponents(std::function<void(state::StateController*)> func) {
+  std::lock_guard<std::recursive_mutex> lock(mutex_);
+  for (auto* component: getAllComponents()) {
+    func(component);
+  }
+}
+
+void FlowController::executeOnComponent(const std::string &name, std::function<void(state::StateController*)> func) {
+  std::lock_guard<std::recursive_mutex> lock(mutex_);
+  if (auto* component = getComponent(name); component != nullptr) {
+    func(component);
+  }

Review comment:
       maybe add an error log if the component is not found?  I think that should never happen, and a log could be useful if it does

##########
File path: libminifi/src/FlowController.cpp
##########
@@ -523,14 +537,14 @@ void FlowController::getAllProcessorControllers(std::vector<state::StateControll
   }
 }
 
-void FlowController::getProcessorController(const std::string& name, std::vector<state::StateController*>& controllerVec,
+void FlowController::getProcessorController(const std::string& name, state::StateController*& controller,
                                           const std::function<std::unique_ptr<state::ProcessorController>(core::Processor&)>& controllerFactory) {
   auto* processor = root_->findProcessorByName(name);
-  auto& controller = processor_to_controller_[processor->getUUID()];
-  if (!controller) {
-    controller = controllerFactory(*processor);
+  auto& foundController = processor_to_controller_[processor->getUUID()];

Review comment:
       we should check if `processor` is null before dereferencing it

##########
File path: libminifi/include/FlowController.h
##########
@@ -234,7 +233,11 @@ class FlowController : public core::controller::ForwardingControllerServiceProvi
   std::chrono::steady_clock::time_point start_time_;
 
  private:
-  void getProcessorController(const std::string& name, std::vector<state::StateController*>& controllerVec,
+  std::vector<state::StateController*> getAllComponents();
+
+  state::StateController* getComponent(const std::string &name);
+
+  void getProcessorController(const std::string& name, state::StateController*& controller,
                               const std::function<std::unique_ptr<state::ProcessorController>(core::Processor&)>& controllerFactory);

Review comment:
       this could return the `controller` instead of using the slightly strange-looking `state::StateController*&` out parameter:
   ```suggestion
     state::StateController* getProcessorController(const std::string& name,
                                 const std::function<std::unique_ptr<state::ProcessorController>(core::Processor&)>& controllerFactory);
   ```

##########
File path: libminifi/src/FlowController.cpp
##########
@@ -523,14 +537,14 @@ void FlowController::getAllProcessorControllers(std::vector<state::StateControll
   }
 }
 
-void FlowController::getProcessorController(const std::string& name, std::vector<state::StateController*>& controllerVec,
+void FlowController::getProcessorController(const std::string& name, state::StateController*& controller,
                                           const std::function<std::unique_ptr<state::ProcessorController>(core::Processor&)>& controllerFactory) {
   auto* processor = root_->findProcessorByName(name);
-  auto& controller = processor_to_controller_[processor->getUUID()];
-  if (!controller) {
-    controller = controllerFactory(*processor);
+  auto& foundController = processor_to_controller_[processor->getUUID()];
+  if (!foundController) {
+    foundController = controllerFactory(*processor);
   }
-  controllerVec.push_back(controller.get());
+  controller = foundController.get();

Review comment:
       `controller` is dangling: `controllerFactory` returns a `unique_ptr` which gets destroyed at the end of the function.
   
   I think we should add the newly created `ProcessorController` to the `processor_to_controller_` map, both here and in `getAllProcessorControllers`.

##########
File path: libminifi/src/c2/ControllerSocketProtocol.cpp
##########
@@ -180,14 +178,18 @@ void ControllerSocketProtocol::initialize(core::controller::ControllerServicePro
             resp.write(response.str());
             stream->write(resp.getBuffer());
           } else if (what == "components") {
+            std::vector<std::pair<std::string, bool>> components;
+            update_sink_->executeOnAllComponents([&components](state::StateController* component){
+              components.emplace_back(component->getComponentName(), component->isRunning());
+            });
             io::BufferStream resp;
             resp.write(&head, 1);
-            const auto size_ = gsl::narrow<uint16_t>(update_sink_->getAllComponents().size());
-            resp.write(size_);
-            for (const auto &component : update_sink_->getAllComponents()) {
-              resp.write(component->getComponentName());
-              resp.write(component->isRunning() ? "true" : "false");
+            resp.write(gsl::narrow<uint16_t>(gsl::narrow<uint16_t>(components.size())));

Review comment:
       one `gsl_narrow` is enough :)
   ```suggestion
               resp.write(gsl::narrow<uint16_t>(components.size()));
   ```




-- 
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 closed pull request #1284: MINIFICPP-1776 - Parallel heartbeat and flow update can cause crash

Posted by GitBox <gi...@apache.org>.
martinzink closed pull request #1284:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1284


   


-- 
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 change in pull request #1284: MINIFICPP-1776 - Parallel heartbeat and flow update can cause crash

Posted by GitBox <gi...@apache.org>.
fgerlits commented on a change in pull request #1284:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1284#discussion_r832018077



##########
File path: libminifi/src/FlowController.cpp
##########
@@ -523,14 +537,14 @@ void FlowController::getAllProcessorControllers(std::vector<state::StateControll
   }
 }
 
-void FlowController::getProcessorController(const std::string& name, std::vector<state::StateController*>& controllerVec,
+void FlowController::getProcessorController(const std::string& name, state::StateController*& controller,
                                           const std::function<std::unique_ptr<state::ProcessorController>(core::Processor&)>& controllerFactory) {
   auto* processor = root_->findProcessorByName(name);
-  auto& controller = processor_to_controller_[processor->getUUID()];
-  if (!controller) {
-    controller = controllerFactory(*processor);
+  auto& foundController = processor_to_controller_[processor->getUUID()];

Review comment:
       we should check if `processor` is null before dereferencing it (I know it was like this before, but we should fix it)




-- 
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 change in pull request #1284: MINIFICPP-1776 - Parallel heartbeat and flow update can cause crash

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #1284:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1284#discussion_r832132536



##########
File path: libminifi/src/FlowController.cpp
##########
@@ -438,30 +432,43 @@ std::shared_ptr<state::response::ResponseNode> FlowController::getAgentManifest(
   return agentInfo;
 }
 
+void FlowController::executeOnAllComponents(std::function<void(state::StateController*)> func) {
+  std::lock_guard<std::recursive_mutex> lock(mutex_);
+  for (auto* component: getAllComponents()) {
+    func(component);
+  }
+}
+
+void FlowController::executeOnComponent(const std::string &name, std::function<void(state::StateController*)> func) {
+  std::lock_guard<std::recursive_mutex> lock(mutex_);
+  if (auto* component = getComponent(name); component != nullptr) {
+    func(component);

Review comment:
       The function could take a reference instead, since the pointer is always valid.

##########
File path: libminifi/src/FlowController.cpp
##########
@@ -438,30 +432,43 @@ std::shared_ptr<state::response::ResponseNode> FlowController::getAgentManifest(
   return agentInfo;
 }
 
+void FlowController::executeOnAllComponents(std::function<void(state::StateController*)> func) {
+  std::lock_guard<std::recursive_mutex> lock(mutex_);
+  for (auto* component: getAllComponents()) {
+    func(component);
+  }

Review comment:
       Here as well




-- 
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 change in pull request #1284: MINIFICPP-1776 - Parallel heartbeat and flow update can cause crash

Posted by GitBox <gi...@apache.org>.
lordgamez commented on a change in pull request #1284:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1284#discussion_r830880280



##########
File path: extensions/http-curl/tests/C2ClearCoreComponentStateTest.cpp
##########
@@ -51,10 +51,21 @@ class VerifyC2ClearCoreComponentState : public VerifyC2Base {
 
  protected:
   void updateProperties(minifi::FlowController& flow_controller) override {
-    dynamic_cast<minifi::state::ProcessorController*>(flow_controller.getComponents("TailFile1")[0])
-        ->getProcessor()->setProperty(minifi::processors::TailFile::FileName, test_file_1_);
-    dynamic_cast<minifi::state::ProcessorController*>(flow_controller.getComponents("TailFile2")[0])
-        ->getProcessor()->setProperty(minifi::processors::TailFile::FileName, test_file_2_);
+    size_t componentIdx = 0;
+    auto setFileName = [&componentIdx] (const std::string& fileName, minifi::state::StateController* component){

Review comment:
       Could you please elaborate on the purpose of the `componentIdx` here? What I see that during the test in both cases it is called with the value 0, why the check and the reset of the counter is needed?




-- 
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 change in pull request #1284: MINIFICPP-1776 - Parallel heartbeat and flow update can cause crash

Posted by GitBox <gi...@apache.org>.
adam-markovics commented on a change in pull request #1284:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1284#discussion_r832043146



##########
File path: libminifi/src/c2/ControllerSocketProtocol.cpp
##########
@@ -180,14 +178,18 @@ void ControllerSocketProtocol::initialize(core::controller::ControllerServicePro
             resp.write(response.str());
             stream->write(resp.getBuffer());
           } else if (what == "components") {
+            std::vector<std::pair<std::string, bool>> components;
+            update_sink_->executeOnAllComponents([&components](state::StateController* component){
+              components.emplace_back(component->getComponentName(), component->isRunning());
+            });
             io::BufferStream resp;
             resp.write(&head, 1);
-            const auto size_ = gsl::narrow<uint16_t>(update_sink_->getAllComponents().size());
-            resp.write(size_);
-            for (const auto &component : update_sink_->getAllComponents()) {
-              resp.write(component->getComponentName());
-              resp.write(component->isRunning() ? "true" : "false");
+            resp.write(gsl::narrow<uint16_t>(gsl::narrow<uint16_t>(components.size())));

Review comment:
       Oops, yes. :)




-- 
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 change in pull request #1284: MINIFICPP-1776 - Parallel heartbeat and flow update can cause crash

Posted by GitBox <gi...@apache.org>.
adam-markovics commented on a change in pull request #1284:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1284#discussion_r830884750



##########
File path: extensions/http-curl/tests/C2ClearCoreComponentStateTest.cpp
##########
@@ -51,10 +51,21 @@ class VerifyC2ClearCoreComponentState : public VerifyC2Base {
 
  protected:
   void updateProperties(minifi::FlowController& flow_controller) override {
-    dynamic_cast<minifi::state::ProcessorController*>(flow_controller.getComponents("TailFile1")[0])
-        ->getProcessor()->setProperty(minifi::processors::TailFile::FileName, test_file_1_);
-    dynamic_cast<minifi::state::ProcessorController*>(flow_controller.getComponents("TailFile2")[0])
-        ->getProcessor()->setProperty(minifi::processors::TailFile::FileName, test_file_2_);
+    size_t componentIdx = 0;
+    auto setFileName = [&componentIdx] (const std::string& fileName, minifi::state::StateController* component){

Review comment:
       It's going to be removed in a future commit.




-- 
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 change in pull request #1284: MINIFICPP-1776 - Parallel heartbeat and flow update can cause crash

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #1284:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1284#discussion_r832050714



##########
File path: libminifi/src/FlowController.cpp
##########
@@ -438,6 +440,20 @@ std::shared_ptr<state::response::ResponseNode> FlowController::getAgentManifest(
   return agentInfo;
 }
 
+void FlowController::executeOnAllComponents(std::function<void(state::StateController*)> func) {
+  std::lock_guard<std::recursive_mutex> lock(mutex_);
+  for (auto* component: getAllComponents()) {
+    func(component);
+  }
+}
+
+void FlowController::executeOnComponent(const std::string &name, std::function<void(state::StateController*)> func) {
+  std::lock_guard<std::recursive_mutex> lock(mutex_);
+  if (auto* component = getComponent(name); component != nullptr) {
+    func(component);
+  }

Review comment:
       You can even leave out the second part, and it means the same.
   ```
   if (auto* const component = getComponent(name)) {
     func(component);
   }
   ```




-- 
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 change in pull request #1284: MINIFICPP-1776 - Parallel heartbeat and flow update can cause crash

Posted by GitBox <gi...@apache.org>.
fgerlits commented on a change in pull request #1284:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1284#discussion_r832075219



##########
File path: libminifi/src/FlowController.cpp
##########
@@ -523,14 +537,14 @@ void FlowController::getAllProcessorControllers(std::vector<state::StateControll
   }
 }
 
-void FlowController::getProcessorController(const std::string& name, std::vector<state::StateController*>& controllerVec,
+void FlowController::getProcessorController(const std::string& name, state::StateController*& controller,
                                           const std::function<std::unique_ptr<state::ProcessorController>(core::Processor&)>& controllerFactory) {
   auto* processor = root_->findProcessorByName(name);
-  auto& controller = processor_to_controller_[processor->getUUID()];
-  if (!controller) {
-    controller = controllerFactory(*processor);
+  auto& foundController = processor_to_controller_[processor->getUUID()];
+  if (!foundController) {
+    foundController = controllerFactory(*processor);
   }
-  controllerVec.push_back(controller.get());
+  controller = foundController.get();

Review comment:
       Yes, you're right, sorry.  Thanks for the new comment, it makes the code 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



[GitHub] [nifi-minifi-cpp] fgerlits commented on a change in pull request #1284: MINIFICPP-1776 - Parallel heartbeat and flow update can cause crash

Posted by GitBox <gi...@apache.org>.
fgerlits commented on a change in pull request #1284:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1284#discussion_r832085417



##########
File path: libminifi/src/FlowController.cpp
##########
@@ -506,31 +513,36 @@ std::map<std::string, std::unique_ptr<io::InputStream>> FlowController::getDebug
   return debug_info;
 }
 
-void FlowController::getAllProcessorControllers(std::vector<state::StateController*>& controllerVec,
-                                              const std::function<std::unique_ptr<state::ProcessorController>(core::Processor&)>& controllerFactory) {
-  std::lock_guard<std::recursive_mutex> lock(mutex_);
-
+std::vector<state::StateController*> FlowController::getAllProcessorControllers(
+        const std::function<std::unique_ptr<state::ProcessorController>(core::Processor&)>& controllerFactory) {
+  std::vector<state::StateController*> controllerVec{this};
   std::vector<core::Processor*> processorVec;
   root_->getAllProcessors(processorVec);
 
   for (const auto& processor : processorVec) {
-    // find controller for processor, if it doesn't exist, create one
+    // reference to the existing or newly created controller
     auto& controller = processor_to_controller_[processor->getUUID()];
     if (!controller) {
       controller = controllerFactory(*processor);
     }
     controllerVec.push_back(controller.get());
   }
+
+  return controllerVec;
 }
 
-void FlowController::getProcessorController(const std::string& name, std::vector<state::StateController*>& controllerVec,
-                                          const std::function<std::unique_ptr<state::ProcessorController>(core::Processor&)>& controllerFactory) {
+state::StateController* FlowController::getProcessorController(const std::string& name, const std::function<std::unique_ptr<state::ProcessorController>(core::Processor&)>& controllerFactory) {
   auto* processor = root_->findProcessorByName(name);
-  auto& controller = processor_to_controller_[processor->getUUID()];
-  if (!controller) {
-    controller = controllerFactory(*processor);
+  if (processor == nullptr) {
+    logger_->log_error("Could not get processor controller for requested name \"%s\", because processor was not found either", name);
+  }

Review comment:
       logging and then crashing is an improvement, but not crashing is even better:
   ```suggestion
     if (processor == nullptr) {
       logger_->log_error("Could not get processor controller for requested name \"%s\", because processor was not found either", name);
       return nullptr;
     }
   ```




-- 
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 change in pull request #1284: MINIFICPP-1776 - Parallel heartbeat and flow update can cause crash

Posted by GitBox <gi...@apache.org>.
adam-markovics commented on a change in pull request #1284:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1284#discussion_r832263059



##########
File path: libminifi/src/FlowController.cpp
##########
@@ -438,30 +432,43 @@ std::shared_ptr<state::response::ResponseNode> FlowController::getAgentManifest(
   return agentInfo;
 }
 
+void FlowController::executeOnAllComponents(std::function<void(state::StateController*)> func) {
+  std::lock_guard<std::recursive_mutex> lock(mutex_);
+  for (auto* component: getAllComponents()) {
+    func(component);
+  }
+}
+
+void FlowController::executeOnComponent(const std::string &name, std::function<void(state::StateController*)> func) {
+  std::lock_guard<std::recursive_mutex> lock(mutex_);
+  if (auto* component = getComponent(name); component != nullptr) {
+    func(component);

Review comment:
       This will result in a large change in all usages, but I'll do it.




-- 
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 change in pull request #1284: MINIFICPP-1776 - Parallel heartbeat and flow update can cause crash

Posted by GitBox <gi...@apache.org>.
adam-markovics commented on a change in pull request #1284:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1284#discussion_r832339844



##########
File path: libminifi/src/FlowController.cpp
##########
@@ -438,30 +432,43 @@ std::shared_ptr<state::response::ResponseNode> FlowController::getAgentManifest(
   return agentInfo;
 }
 
+void FlowController::executeOnAllComponents(std::function<void(state::StateController*)> func) {
+  std::lock_guard<std::recursive_mutex> lock(mutex_);
+  for (auto* component: getAllComponents()) {
+    func(component);
+  }
+}
+
+void FlowController::executeOnComponent(const std::string &name, std::function<void(state::StateController*)> func) {
+  std::lock_guard<std::recursive_mutex> lock(mutex_);
+  if (auto* component = getComponent(name); component != nullptr) {
+    func(component);

Review comment:
       Done.




-- 
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 change in pull request #1284: MINIFICPP-1776 - Parallel heartbeat and flow update can cause crash

Posted by GitBox <gi...@apache.org>.
adam-markovics commented on a change in pull request #1284:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1284#discussion_r832048188



##########
File path: libminifi/src/FlowController.cpp
##########
@@ -438,6 +440,20 @@ std::shared_ptr<state::response::ResponseNode> FlowController::getAgentManifest(
   return agentInfo;
 }
 
+void FlowController::executeOnAllComponents(std::function<void(state::StateController*)> func) {
+  std::lock_guard<std::recursive_mutex> lock(mutex_);
+  for (auto* component: getAllComponents()) {
+    func(component);
+  }
+}
+
+void FlowController::executeOnComponent(const std::string &name, std::function<void(state::StateController*)> func) {
+  std::lock_guard<std::recursive_mutex> lock(mutex_);
+  if (auto* component = getComponent(name); component != nullptr) {
+    func(component);
+  }

Review comment:
       Done.




-- 
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 change in pull request #1284: MINIFICPP-1776 - Parallel heartbeat and flow update can cause crash

Posted by GitBox <gi...@apache.org>.
adam-markovics commented on a change in pull request #1284:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1284#discussion_r832060622



##########
File path: libminifi/include/FlowController.h
##########
@@ -234,7 +233,11 @@ class FlowController : public core::controller::ForwardingControllerServiceProvi
   std::chrono::steady_clock::time_point start_time_;
 
  private:
-  void getProcessorController(const std::string& name, std::vector<state::StateController*>& controllerVec,
+  std::vector<state::StateController*> getAllComponents();
+
+  state::StateController* getComponent(const std::string &name);
+
+  void getProcessorController(const std::string& name, state::StateController*& controller,
                               const std::function<std::unique_ptr<state::ProcessorController>(core::Processor&)>& controllerFactory);

Review comment:
       Done.




-- 
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 change in pull request #1284: MINIFICPP-1776 - Parallel heartbeat and flow update can cause crash

Posted by GitBox <gi...@apache.org>.
fgerlits commented on a change in pull request #1284:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1284#discussion_r832075219



##########
File path: libminifi/src/FlowController.cpp
##########
@@ -523,14 +537,14 @@ void FlowController::getAllProcessorControllers(std::vector<state::StateControll
   }
 }
 
-void FlowController::getProcessorController(const std::string& name, std::vector<state::StateController*>& controllerVec,
+void FlowController::getProcessorController(const std::string& name, state::StateController*& controller,
                                           const std::function<std::unique_ptr<state::ProcessorController>(core::Processor&)>& controllerFactory) {
   auto* processor = root_->findProcessorByName(name);
-  auto& controller = processor_to_controller_[processor->getUUID()];
-  if (!controller) {
-    controller = controllerFactory(*processor);
+  auto& foundController = processor_to_controller_[processor->getUUID()];
+  if (!foundController) {
+    foundController = controllerFactory(*processor);
   }
-  controllerVec.push_back(controller.get());
+  controller = foundController.get();

Review comment:
       Yes, you're right, sorry.




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