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

[GitHub] [nifi-minifi-cpp] lordgamez opened a new pull request, #1568: MINIFICPP-2112 Fix flow update and restart with minifi controller

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

   https://issues.apache.org/jira/browse/MINIFICPP-2112
   
   ---------------
   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 #1568: MINIFICPP-2112 Fix flow update and restart with minifi controller

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


##########
controller/tests/ControllerTests.cpp:
##########
@@ -314,14 +314,14 @@ TEST_CASE_METHOD(ControllerTestFixture, "Test listComponents", "[controllerTests
   }
 
   using org::apache::nifi::minifi::utils::verifyEventHappenedInPollTime;
-  REQUIRE(verifyEventHappenedInPollTime(500ms, [&] { return controller_->isRunning(); }, 20ms));
+  REQUIRE(verifyEventHappenedInPollTime(5s, [&] { return controller_->isRunning(); }, 20ms));

Review Comment:
   Sometimes these checks were flaky in the CI probably due to the introduced communication of the command handlings threads through the `ConcurrentQueue`. 10 times is probably an overkill as it should probably be under 2 seconds in the CI as well, but I wanted to make sure to remove the flakyness and only test the functionality.



-- 
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 #1568: MINIFICPP-2112 Fix flow update and restart with minifi controller

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


##########
libminifi/include/c2/ControllerSocketProtocol.h:
##########
@@ -61,6 +66,46 @@ class ControllerSocketProtocol {
   std::weak_ptr<ControllerSocketReporter> controller_socket_reporter_;
   std::shared_ptr<Configure> configuration_;
   std::shared_ptr<core::logging::Logger> logger_ = core::logging::LoggerFactory<ControllerSocketProtocol>::getLogger();
+  std::mutex initialization_mutex_;
+
+  // Some commands need to restart the controller socket to reinitialize the socket with new data for example new SSL data in case of a flow update
+  // These commands are handled on a separate thread, and while these commands are handled other incoming commands are dropped
+  class SocketRestartCommandProcessor {
+   public:
+    explicit SocketRestartCommandProcessor(state::StateMonitor& update_sink_);
+    ~SocketRestartCommandProcessor();
+
+    enum class Command {
+      FLOW_UPDATE,
+      START
+    };
+
+    struct CommandData {
+      Command command;
+      std::string data;
+    };
+
+    void enqueue(const CommandData& command_data) {
+      is_socket_restarting_ = true;
+      command_queue_.enqueue(command_data);
+      command_queue_condition_variable_.notify_all();
+    }
+
+    bool isSocketRestarting() const {
+      return is_socket_restarting_;
+    }
+
+   private:
+    mutable std::atomic_bool is_socket_restarting_ = false;
+    state::StateMonitor& update_sink_;
+    std::thread command_processor_thread_;
+    std::mutex cv_mutex_;
+    std::condition_variable command_queue_condition_variable_;
+    std::atomic_bool running_ = true;
+    moodycamel::ConcurrentQueue<CommandData> command_queue_;

Review Comment:
   Thanks for the detailed description, good to know these differences. I replaced the in `moodycamel::ConcurrentQueue` with the `utils::ConditionConcurrentQueue` in 87bce05ddabb9a2ee89ecac9681c123c0adbb17a



##########
cmake/MiNiFiOptions.cmake:
##########
@@ -120,6 +120,8 @@ add_minifi_option(ENABLE_KUBERNETES "Enables the Kubernetes extensions." ON)
 add_minifi_option(ENABLE_TEST_PROCESSORS "Enables test processors" OFF)
 add_minifi_option(ENABLE_PROMETHEUS "Enables Prometheus support." ON)
 add_minifi_option(DISABLE_JEMALLOC "Disables jemalloc." OFF)
+add_minifi_option(DISABLE_CURL "Disables curl." OFF)
+add_minifi_option(DISABLE_CONTROLLER "Disables build of MiNiFi controller binary." OFF)

Review Comment:
   These options were present already so I didn't want to change them as they may break our builds. The options are only added here to be available for the docker builds as well.



##########
docker/test/integration/features/minifi_controller.feature:
##########
@@ -0,0 +1,76 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+Feature: MiNiFi Controller functionalities
+  Test MiNiFi Controller functionalities
+
+  Background:
+    Given the content of "/tmp/output" is monitored
+
+  Scenario: Flow config can be updated through MiNiFi controller
+    Given a GenerateFlowFile processor
+    And a file with the content "test" is present in "/tmp/input"
+    And controller socket properties are set up
+    When all instances start up
+    And MiNiFi config is updated through MiNiFi controller
+    Then a flowfile with the content "test" is placed in the monitored directory in less than 60 seconds
+    And the updated config is persisted
+
+  Scenario: A component can be stopped
+    Given a GenerateFlowFile processor
+    And a file with the content "test" is present in "/tmp/input"
+    And controller socket properties are set up
+    When all instances start up
+    And the GenerateFlowFile component is stopped through MiNiFi controller
+    Then the GenerateFlowFile component is not running
+    And the FlowController component is running
+
+  Scenario: If FlowController is stopped all other components are stopped
+    Given a GenerateFlowFile processor
+    And a file with the content "test" is present in "/tmp/input"
+    And controller socket properties are set up
+    When all instances start up
+    And the FlowController component is stopped through MiNiFi controller
+    And the FlowController component is started through MiNiFi controller
+    Then the GenerateFlowFile component is not running
+    And the FlowController component is not running
+
+  Scenario: FlowController can be stopped and restarted
+    Given a GenerateFlowFile processor
+    And a file with the content "test" is present in "/tmp/input"
+    And controller socket properties are set up
+    When all instances start up
+    And the FlowController component is stopped through MiNiFi controller
+    And the FlowController component is started through MiNiFi controller
+    Then the GenerateFlowFile component is running
+    And the FlowController component is running

Review Comment:
   Good catch! As the start operation is not immediate the previous test could still pass with the negative checks before the start operation was finished. I fixed the test in 87bce05ddabb9a2ee89ecac9681c123c0adbb17a



-- 
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 #1568: MINIFICPP-2112 Fix flow update and restart with minifi controller

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


##########
cmake/DockerConfig.cmake:
##########
@@ -47,8 +47,9 @@ add_custom_target(
              -DENABLE_ELASTICSEARCH=OFF
              -DENABLE_LUA_SCRIPTING=OFF
              -DENABLE_PYTHON_SCRIPTING=OFF
-	     -DENABLE_OPC=OFF
-             -DENABLE_ENCRYPT_CONFIG=OFF \"
+             -DENABLE_OPC=OFF
+             -DENABLE_ENCRYPT_CONFIG=OFF
+             -DDISABLE_CONTROLLER=ON \"

Review Comment:
   That makes sense, added in 765bea7fbb282048bc9c3a223f667b37620e6039



##########
controller/tests/ControllerTests.cpp:
##########
@@ -314,14 +314,14 @@ TEST_CASE_METHOD(ControllerTestFixture, "Test listComponents", "[controllerTests
   }
 
   using org::apache::nifi::minifi::utils::verifyEventHappenedInPollTime;
-  REQUIRE(verifyEventHappenedInPollTime(500ms, [&] { return controller_->isRunning(); }, 20ms));
+  REQUIRE(verifyEventHappenedInPollTime(5s, [&] { return controller_->isRunning(); }, 20ms));

Review Comment:
   Sometimes these checks were flaky in the CI probably due to the introduced communication of the command handlings threads through the `ConcurrentQueue`. 10 times is probably an overkill as it should probably be under 2 seconds in the CI as well, but I wanted to make sure to remove the flakyness and only that the functionality.



-- 
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 #1568: MINIFICPP-2112 Fix flow update and restart with minifi controller

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


##########
libminifi/include/c2/ControllerSocketProtocol.h:
##########
@@ -61,6 +66,46 @@ class ControllerSocketProtocol {
   std::weak_ptr<ControllerSocketReporter> controller_socket_reporter_;
   std::shared_ptr<Configure> configuration_;
   std::shared_ptr<core::logging::Logger> logger_ = core::logging::LoggerFactory<ControllerSocketProtocol>::getLogger();
+  std::mutex initialization_mutex_;
+
+  // Some commands need to restart the controller socket to reinitialize the socket with new data for example new SSL data in case of a flow update
+  // These commands are handled on a separate thread, and while these commands are handled other incoming commands are dropped
+  class SocketRestartCommandProcessor {
+   public:
+    explicit SocketRestartCommandProcessor(state::StateMonitor& update_sink_);
+    ~SocketRestartCommandProcessor();
+
+    enum class Command {
+      FLOW_UPDATE,
+      START
+    };
+
+    struct CommandData {
+      Command command;
+      std::string data;
+    };
+
+    void enqueue(const CommandData& command_data) {
+      is_socket_restarting_ = true;
+      command_queue_.enqueue(command_data);
+      command_queue_condition_variable_.notify_all();
+    }
+
+    bool isSocketRestarting() const {
+      return is_socket_restarting_;
+    }
+
+   private:
+    mutable std::atomic_bool is_socket_restarting_ = false;
+    state::StateMonitor& update_sink_;
+    std::thread command_processor_thread_;
+    std::mutex cv_mutex_;
+    std::condition_variable command_queue_condition_variable_;
+    std::atomic_bool running_ = true;
+    moodycamel::ConcurrentQueue<CommandData> command_queue_;

Review Comment:
   The minifi `utils::ConditionConcurrentQueue` could be a better alternative here. It's not as performant in high load scenarios, but performance doesn't really matter here, and it's less likely to change the order of commands, and it has builtin waiting dequeue functions.
   
   `moodycamel::ConcurrentQueue` works with per-thread arrays, and the consumer always picks the first non-empty thread array. This means that it doesn't keep any ordering between different sources, and it may return elements that were inserted later if they happen to be in the first bucket/array. Strictly speaking, this still gives happens-before guarantees within a producer thread, but nothing between threads. If the second thread inserted something much earlier, it will still return the item from the first thread if there are any. We used to have strange priority and CPU starving issues when we used to use this queue for scheduling processors. It's unlikely to cause any issues here, other than occasional command reordering between multiple independent clients.
   
   https://github.com/cameron314/concurrentqueue#high-level-design



-- 
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 #1568: MINIFICPP-2112 Fix flow update and restart with minifi controller

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


##########
libminifi/src/core/ProcessGroup.cpp:
##########
@@ -133,6 +133,7 @@ void ProcessGroup::startProcessingProcessors(TimerDrivenSchedulingAgent& timeSch
   for (const auto processor : failed_processors_) {
     try {
       logger_->log_debug("Starting %s", processor->getName());
+      processor->setScheduledState(core::ScheduledState::RUNNING);

Review Comment:
   Scheduling state needs to be reset to RUNNING if FlowController is stopped through controller or C2 otherwise scheduler will not be able to restart 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] lordgamez commented on a diff in pull request #1568: MINIFICPP-2112 Fix flow update and restart with minifi controller

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


##########
libminifi/include/FlowController.h:
##########
@@ -149,6 +149,22 @@ class FlowController : public core::controller::ForwardingControllerServiceProvi
   std::map<std::string, std::unique_ptr<io::InputStream>> getDebugInfo() override;
 
  private:
+  class UpdateState {
+   public:
+    void beginUpdate() {
+      ++update_block_count_;
+    }
+    void endUpdate() {
+      --update_block_count_;
+    }
+    bool isUpdating() const {
+      return update_block_count_ > 0;
+    }
+
+   private:
+    std::atomic<uint32_t> update_block_count_;
+  };
+

Review Comment:
   Good point, added a scoped UpdateLock in ffefd48806fffcc07dbbff20d1159e206e19824f



##########
docker/test/integration/features/CMakeLists.txt:
##########
@@ -87,6 +87,10 @@ if (ENABLE_PROMETHEUS)
     set(ENABLED_BEHAVE_TESTS "${ENABLED_BEHAVE_TESTS};${CMAKE_CURRENT_SOURCE_DIR}/prometheus.feature")
 endif()
 
+if (NOT DISABLE_CURL AND NOT DISABLE_CONTROLLER)
+    set(ENABLED_BEHAVE_TESTS "${ENABLED_BEHAVE_TESTS};${CMAKE_CURRENT_SOURCE_DIR}/minifi_controller.feature")

Review Comment:
   Updated in ffefd48806fffcc07dbbff20d1159e206e19824f



-- 
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 #1568: MINIFICPP-2112 Fix flow update and restart with minifi controller

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


##########
libminifi/src/FlowController.cpp:
##########
@@ -127,35 +127,34 @@ bool FlowController::applyConfiguration(const std::string &source, const std::st
 
   logger_->log_info("Starting to reload Flow Controller with flow control name %s, version %d", newRoot->getName(), newRoot->getVersion());
 
-  updating_.beginUpdate();
   bool started = false;
-
   {
-    std::lock_guard<std::recursive_mutex> flow_lock(mutex_);
-    stop();
-
-    root_wrapper_.setNewRoot(std::move(newRoot));
-    initialized_ = false;
-    try {
-      load(true);
-      started = start() == 0;
-    } catch (const std::exception& ex) {
-      logger_->log_error("Caught exception while starting flow, type %s, what: %s", typeid(ex).name(), ex.what());
-    } catch (...) {
-      logger_->log_error("Caught unknown exception while starting flow, type %s", getCurrentExceptionTypeName());
-    }
-    if (!started) {
-      logger_->log_error("Failed to start new flow, restarting previous flow");
-      root_wrapper_.restoreBackup();
-      load(true);
-      start();
-    } else {
-      root_wrapper_.clearBackup();
+    auto update_lock = updating_.getUpdateLock();
+    {
+      std::lock_guard<std::recursive_mutex> flow_lock(mutex_);

Review Comment:
   Updated in 1fd1685fbccb60386133baeee87ddcffa4747487



##########
libminifi/include/FlowController.h:
##########
@@ -149,6 +149,34 @@ class FlowController : public core::controller::ForwardingControllerServiceProvi
   std::map<std::string, std::unique_ptr<io::InputStream>> getDebugInfo() override;
 
  private:
+  class UpdateState {
+    class UpdateLock;
+   public:
+    UpdateLock getUpdateLock() { return UpdateLock(*this); }
+    bool isUpdating() const { return update_block_count_ > 0; }
+
+   private:
+    class UpdateLock {
+     public:
+      UpdateLock(UpdateState& update_state) : update_state_(update_state) {
+        update_state_.beginUpdate();
+      }
+
+      ~UpdateLock() {
+        update_state_.endUpdate();
+      }
+     private:
+      UpdateState& update_state_;
+    };
+

Review Comment:
   The latter sounds good, thanks, updated in 1fd1685fbccb60386133baeee87ddcffa4747487



-- 
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 #1568: MINIFICPP-2112 Fix flow update and restart with minifi controller

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


##########
docker/test/integration/features/CMakeLists.txt:
##########
@@ -87,6 +87,10 @@ if (ENABLE_PROMETHEUS)
     set(ENABLED_BEHAVE_TESTS "${ENABLED_BEHAVE_TESTS};${CMAKE_CURRENT_SOURCE_DIR}/prometheus.feature")
 endif()
 
+if (NOT DISABLE_CURL AND NOT DISABLE_CONTROLLER)
+    set(ENABLED_BEHAVE_TESTS "${ENABLED_BEHAVE_TESTS};${CMAKE_CURRENT_SOURCE_DIR}/minifi_controller.feature")

Review Comment:
   `list(APPEND` would be more descriptive



##########
libminifi/include/FlowController.h:
##########
@@ -149,6 +149,22 @@ class FlowController : public core::controller::ForwardingControllerServiceProvi
   std::map<std::string, std::unique_ptr<io::InputStream>> getDebugInfo() override;
 
  private:
+  class UpdateState {
+   public:
+    void beginUpdate() {
+      ++update_block_count_;
+    }
+    void endUpdate() {
+      --update_block_count_;
+    }
+    bool isUpdating() const {
+      return update_block_count_ > 0;
+    }
+
+   private:
+    std::atomic<uint32_t> update_block_count_;
+  };
+

Review Comment:
   A scoped solution would be better in that it's impossible to forget closing an opened update block. Maybe we could even use std::scoped_lock to track update blocks.



-- 
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 #1568: MINIFICPP-2112 Fix flow update and restart with minifi controller

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


##########
libminifi/include/FlowController.h:
##########
@@ -149,6 +149,34 @@ class FlowController : public core::controller::ForwardingControllerServiceProvi
   std::map<std::string, std::unique_ptr<io::InputStream>> getDebugInfo() override;
 
  private:
+  class UpdateState {
+    class UpdateLock;
+   public:
+    UpdateLock getUpdateLock() { return UpdateLock(*this); }
+    bool isUpdating() const { return update_block_count_ > 0; }
+
+   private:
+    class UpdateLock {
+     public:
+      UpdateLock(UpdateState& update_state) : update_state_(update_state) {
+        update_state_.beginUpdate();
+      }
+
+      ~UpdateLock() {
+        update_state_.endUpdate();
+      }
+     private:
+      UpdateState& update_state_;
+    };
+

Review Comment:
   Please check the rule of 5 for this class. I think copying the object will result in double decrease and underflowing update count.



-- 
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 #1568: MINIFICPP-2112 Fix flow update and restart with minifi controller

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


##########
libminifi/src/FlowController.cpp:
##########
@@ -193,7 +192,6 @@ int16_t FlowController::stop() {
     this->content_repo_->stop();
     // stop the ControllerServices
     disableAllControllerServices();
-    initialized_ = false;

Review Comment:
   Initialized state should only be false when we do not have a valid flow loaded.



-- 
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 #1568: MINIFICPP-2112 Fix flow update and restart with minifi controller

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


##########
libminifi/include/FlowController.h:
##########
@@ -149,6 +149,34 @@ class FlowController : public core::controller::ForwardingControllerServiceProvi
   std::map<std::string, std::unique_ptr<io::InputStream>> getDebugInfo() override;
 
  private:
+  class UpdateState {
+    class UpdateLock;
+   public:
+    UpdateLock getUpdateLock() { return UpdateLock(*this); }
+    bool isUpdating() const { return update_block_count_ > 0; }
+
+   private:
+    class UpdateLock {
+     public:
+      UpdateLock(UpdateState& update_state) : update_state_(update_state) {
+        update_state_.beginUpdate();
+      }
+
+      ~UpdateLock() {
+        update_state_.endUpdate();
+      }
+     private:
+      UpdateState& update_state_;
+    };
+

Review Comment:
   Please check the rule of 5 for this class. I think copying the object will result in double decrease and underflowing update count.
   
   Alternatively, you could keep the `beginUpdate`/`endUpdate` functionality public, add aliases as `lock`/`unlock`, and use `std::scoped_lock` on the `UpdateState`



-- 
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 closed pull request #1568: MINIFICPP-2112 Fix flow update and restart with minifi controller

Posted by "szaszm (via GitHub)" <gi...@apache.org>.
szaszm closed pull request #1568: MINIFICPP-2112 Fix flow update and restart with minifi controller
URL: https://github.com/apache/nifi-minifi-cpp/pull/1568


-- 
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 #1568: MINIFICPP-2112 Fix flow update and restart with minifi controller

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


##########
cmake/MiNiFiOptions.cmake:
##########
@@ -120,6 +120,8 @@ add_minifi_option(ENABLE_KUBERNETES "Enables the Kubernetes extensions." ON)
 add_minifi_option(ENABLE_TEST_PROCESSORS "Enables test processors" OFF)
 add_minifi_option(ENABLE_PROMETHEUS "Enables Prometheus support." ON)
 add_minifi_option(DISABLE_JEMALLOC "Disables jemalloc." OFF)
+add_minifi_option(DISABLE_CURL "Disables curl." OFF)
+add_minifi_option(DISABLE_CONTROLLER "Disables build of MiNiFi controller binary." OFF)

Review Comment:
   could we flip these to "ENABLE_*"?



-- 
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 #1568: MINIFICPP-2112 Fix flow update and restart with minifi controller

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


##########
libminifi/include/c2/ControllerSocketProtocol.h:
##########
@@ -61,6 +66,46 @@ class ControllerSocketProtocol {
   std::weak_ptr<ControllerSocketReporter> controller_socket_reporter_;
   std::shared_ptr<Configure> configuration_;
   std::shared_ptr<core::logging::Logger> logger_ = core::logging::LoggerFactory<ControllerSocketProtocol>::getLogger();
+  std::mutex initialization_mutex_;
+
+  // Some commands need to restart the controller socket to reinitialize the socket with new data for example new SSL data in case of a flow update
+  // These commands are handled on a separate thread, and while these commands are handled other incoming commands are dropped
+  class SocketRestartCommandProcessor {

Review Comment:
   The reason the controller socket is inside the FlowController is because it can be effected by the flow update. If the controller socket uses SSL the SSL context may be read from the flow configuration file. If the flow config changes the socket has to be restarted in case there is a change in the used SSLContextService.



-- 
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 #1568: MINIFICPP-2112 Fix flow update and restart with minifi controller

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


##########
cmake/MiNiFiOptions.cmake:
##########
@@ -120,6 +120,8 @@ add_minifi_option(ENABLE_KUBERNETES "Enables the Kubernetes extensions." ON)
 add_minifi_option(ENABLE_TEST_PROCESSORS "Enables test processors" OFF)
 add_minifi_option(ENABLE_PROMETHEUS "Enables Prometheus support." ON)
 add_minifi_option(DISABLE_JEMALLOC "Disables jemalloc." OFF)
+add_minifi_option(DISABLE_CURL "Disables curl." OFF)
+add_minifi_option(DISABLE_CONTROLLER "Disables build of MiNiFi controller binary." OFF)

Review Comment:
   I see, thanks for clarifying 



-- 
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 #1568: MINIFICPP-2112 Fix flow update and restart with minifi controller

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


##########
libminifi/src/FlowController.cpp:
##########
@@ -193,7 +192,6 @@ int16_t FlowController::stop() {
     this->content_repo_->stop();
     // stop the ControllerServices
     disableAllControllerServices();
-    initialized_ = false;

Review Comment:
   Note: Initialized state should only be false when we do not have a valid flow loaded.



-- 
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 #1568: MINIFICPP-2112 Fix flow update and restart with minifi controller

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


##########
controller/tests/ControllerTests.cpp:
##########
@@ -314,14 +314,14 @@ TEST_CASE_METHOD(ControllerTestFixture, "Test listComponents", "[controllerTests
   }
 
   using org::apache::nifi::minifi::utils::verifyEventHappenedInPollTime;
-  REQUIRE(verifyEventHappenedInPollTime(500ms, [&] { return controller_->isRunning(); }, 20ms));
+  REQUIRE(verifyEventHappenedInPollTime(5s, [&] { return controller_->isRunning(); }, 20ms));

Review Comment:
   Why are these timeouts extended 10x?



##########
cmake/DockerConfig.cmake:
##########
@@ -47,8 +47,9 @@ add_custom_target(
              -DENABLE_ELASTICSEARCH=OFF
              -DENABLE_LUA_SCRIPTING=OFF
              -DENABLE_PYTHON_SCRIPTING=OFF
-	     -DENABLE_OPC=OFF
-             -DENABLE_ENCRYPT_CONFIG=OFF \"
+             -DENABLE_OPC=OFF
+             -DENABLE_ENCRYPT_CONFIG=OFF
+             -DDISABLE_CONTROLLER=ON \"

Review Comment:
   If the size difference is 1-2% max., then I would keep the controller inside the container. The debugging possibilities through `docker exec` are worth a minor overhead IMO.



-- 
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 #1568: MINIFICPP-2112 Fix flow update and restart with minifi controller

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


##########
libminifi/include/FlowController.h:
##########
@@ -149,6 +149,34 @@ class FlowController : public core::controller::ForwardingControllerServiceProvi
   std::map<std::string, std::unique_ptr<io::InputStream>> getDebugInfo() override;
 
  private:
+  class UpdateState {
+    class UpdateLock;
+   public:
+    UpdateLock getUpdateLock() { return UpdateLock(*this); }
+    bool isUpdating() const { return update_block_count_ > 0; }
+
+   private:
+    class UpdateLock {
+     public:
+      UpdateLock(UpdateState& update_state) : update_state_(update_state) {
+        update_state_.beginUpdate();
+      }
+
+      ~UpdateLock() {
+        update_state_.endUpdate();
+      }
+     private:
+      UpdateState& update_state_;
+    };
+

Review Comment:
   Please check the rule of 5 for this class. I think copying the object will result in double decrease and underflowing update count.
   
   Alternatively, you could remove this class, keep the `beginUpdate`/`endUpdate` functionality public, add aliases as `lock`/`unlock`, and use `std::scoped_lock` on the `UpdateState`



-- 
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 #1568: MINIFICPP-2112 Fix flow update and restart with minifi controller

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


##########
libminifi/include/c2/ControllerSocketProtocol.h:
##########
@@ -61,6 +66,46 @@ class ControllerSocketProtocol {
   std::weak_ptr<ControllerSocketReporter> controller_socket_reporter_;
   std::shared_ptr<Configure> configuration_;
   std::shared_ptr<core::logging::Logger> logger_ = core::logging::LoggerFactory<ControllerSocketProtocol>::getLogger();
+  std::mutex initialization_mutex_;
+
+  // Some commands need to restart the controller socket to reinitialize the socket with new data for example new SSL data in case of a flow update
+  // These commands are handled on a separate thread, and while these commands are handled other incoming commands are dropped
+  class SocketRestartCommandProcessor {

Review Comment:
   The reason the controller socket is inside the FlowController is because it can be effected by the flow update. If the controller socket uses SSL the SSL context may be read from the flow configuration file. If a the flow config changes it has to be restarted in case there is a change in the used SSLContextService.



-- 
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 #1568: MINIFICPP-2112 Fix flow update and restart with minifi controller

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


##########
libminifi/include/c2/ControllerSocketProtocol.h:
##########
@@ -61,6 +66,46 @@ class ControllerSocketProtocol {
   std::weak_ptr<ControllerSocketReporter> controller_socket_reporter_;
   std::shared_ptr<Configure> configuration_;
   std::shared_ptr<core::logging::Logger> logger_ = core::logging::LoggerFactory<ControllerSocketProtocol>::getLogger();
+  std::mutex initialization_mutex_;
+
+  // Some commands need to restart the controller socket to reinitialize the socket with new data for example new SSL data in case of a flow update
+  // These commands are handled on a separate thread, and while these commands are handled other incoming commands are dropped
+  class SocketRestartCommandProcessor {

Review Comment:
   I think a more appropriate fix would be moving the controller socket out of FlowController, so it's unaffected by flow update and restart commands.



-- 
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 #1568: MINIFICPP-2112 Fix flow update and restart with minifi controller

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


##########
docker/test/integration/features/minifi_controller.feature:
##########
@@ -0,0 +1,76 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+Feature: MiNiFi Controller functionalities
+  Test MiNiFi Controller functionalities
+
+  Background:
+    Given the content of "/tmp/output" is monitored
+
+  Scenario: Flow config can be updated through MiNiFi controller
+    Given a GenerateFlowFile processor
+    And a file with the content "test" is present in "/tmp/input"
+    And controller socket properties are set up
+    When all instances start up
+    And MiNiFi config is updated through MiNiFi controller
+    Then a flowfile with the content "test" is placed in the monitored directory in less than 60 seconds
+    And the updated config is persisted
+
+  Scenario: A component can be stopped
+    Given a GenerateFlowFile processor
+    And a file with the content "test" is present in "/tmp/input"
+    And controller socket properties are set up
+    When all instances start up
+    And the GenerateFlowFile component is stopped through MiNiFi controller
+    Then the GenerateFlowFile component is not running
+    And the FlowController component is running
+
+  Scenario: If FlowController is stopped all other components are stopped
+    Given a GenerateFlowFile processor
+    And a file with the content "test" is present in "/tmp/input"
+    And controller socket properties are set up
+    When all instances start up
+    And the FlowController component is stopped through MiNiFi controller
+    And the FlowController component is started through MiNiFi controller
+    Then the GenerateFlowFile component is not running
+    And the FlowController component is not running
+
+  Scenario: FlowController can be stopped and restarted
+    Given a GenerateFlowFile processor
+    And a file with the content "test" is present in "/tmp/input"
+    And controller socket properties are set up
+    When all instances start up
+    And the FlowController component is stopped through MiNiFi controller
+    And the FlowController component is started through MiNiFi controller
+    Then the GenerateFlowFile component is running
+    And the FlowController component is running

Review Comment:
   this seems to do the same setup as the previous one, yet it expects the opposite result



-- 
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 #1568: MINIFICPP-2112 Fix flow update and restart with minifi controller

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


##########
libminifi/src/FlowController.cpp:
##########
@@ -127,35 +127,34 @@ bool FlowController::applyConfiguration(const std::string &source, const std::st
 
   logger_->log_info("Starting to reload Flow Controller with flow control name %s, version %d", newRoot->getName(), newRoot->getVersion());
 
-  updating_.beginUpdate();
   bool started = false;
-
   {
-    std::lock_guard<std::recursive_mutex> flow_lock(mutex_);
-    stop();
-
-    root_wrapper_.setNewRoot(std::move(newRoot));
-    initialized_ = false;
-    try {
-      load(true);
-      started = start() == 0;
-    } catch (const std::exception& ex) {
-      logger_->log_error("Caught exception while starting flow, type %s, what: %s", typeid(ex).name(), ex.what());
-    } catch (...) {
-      logger_->log_error("Caught unknown exception while starting flow, type %s", getCurrentExceptionTypeName());
-    }
-    if (!started) {
-      logger_->log_error("Failed to start new flow, restarting previous flow");
-      root_wrapper_.restoreBackup();
-      load(true);
-      start();
-    } else {
-      root_wrapper_.clearBackup();
+    auto update_lock = updating_.getUpdateLock();
+    {
+      std::lock_guard<std::recursive_mutex> flow_lock(mutex_);

Review Comment:
   The inner scope is not necessary here, the two locks could just follow each other.



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