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 2024/01/26 11:04:22 UTC

[PR] MINIFICPP-2277 Add virtualenv support for python processors [nifi-minifi-cpp]

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

   https://issues.apache.org/jira/browse/MINIFICPP-2277
   
   ---------------
   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


Re: [PR] MINIFICPP-2277 Add virtualenv support for python processors [nifi-minifi-cpp]

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


##########
extensions/python/PythonConfigState.h:
##########
@@ -0,0 +1,50 @@
+/**
+ *
+ * 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.
+ */
+
+#pragma once
+
+#include <filesystem>
+#include <string>
+
+namespace org::apache::nifi::minifi::extensions::python {
+
+struct PythonConfigState {
+ public:
+  PythonConfigState(PythonConfigState&&) = delete;
+  PythonConfigState(const PythonConfigState&) = delete;
+  PythonConfigState& operator=(PythonConfigState&&) = delete;
+  PythonConfigState& operator=(const PythonConfigState&) = delete;
+
+  bool isPackageInstallationNeeded() const {
+    return install_python_packages_automatically && !virtualenv_path.empty();
+  }
+
+  static PythonConfigState& getInstance() {
+    static PythonConfigState config;
+    return config;
+  }

Review Comment:
   It makes sense in the context of this PR, I'm not sure what was the original intention of the `PythonScriptEngine::initialize` static function, as it was previously called with and empty bode from the `PythonCreator`, so my thought was that it was supposed to implement the global python environment specific initialization, but it can be moved to the `PythonCreator` or a separate class.
   
   The problem is that in the PR depending on this https://github.com/apache/nifi-minifi-cpp/pull/1727 we introduce the support for inline python dependency installation, which is separately done for every NiFi python processor initialization that is called from the `ExecutePythonProcessor`. We also only want to initialize the processors that are part of the minifi flow config in the future. The inline python dependency installation needs the information stored in the `PythonConfigState` that is why it needs to be reached from outside the `PythonCreator`



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


Re: [PR] MINIFICPP-2277 Add virtualenv support for python processors [nifi-minifi-cpp]

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


##########
extensions/python/PythonConfigState.h:
##########
@@ -0,0 +1,50 @@
+/**
+ *
+ * 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.
+ */
+
+#pragma once
+
+#include <filesystem>
+#include <string>
+
+namespace org::apache::nifi::minifi::extensions::python {
+
+struct PythonConfigState {
+ public:
+  PythonConfigState(PythonConfigState&&) = delete;
+  PythonConfigState(const PythonConfigState&) = delete;
+  PythonConfigState& operator=(PythonConfigState&&) = delete;
+  PythonConfigState& operator=(const PythonConfigState&) = delete;
+
+  bool isPackageInstallationNeeded() const {
+    return install_python_packages_automatically && !virtualenv_path.empty();
+  }
+
+  static PythonConfigState& getInstance() {
+    static PythonConfigState config;
+    return config;
+  }

Review Comment:
   I think it could be solved by returning or moving the list of dependencies from the python processor to the execution environment class, which can install them as needed. A callback could probably work.



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


Re: [PR] MINIFICPP-2277 Add virtualenv support for python processors [nifi-minifi-cpp]

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


##########
encrypt-config/tests/ConfigFileEncryptorTests.cpp:
##########
@@ -77,7 +77,7 @@ TEST_CASE("ConfigFileEncryptor can encrypt the sensitive properties", "[encrypt-
     uint32_t num_properties_encrypted = encryptSensitivePropertiesInFile(test_file, KEY);
 
     REQUIRE(num_properties_encrypted == 1);
-    REQUIRE(test_file.size() == 110);
+    REQUIRE(test_file.size() == 115);

Review Comment:
   When the default minifi.properties file changes this test always needs to be updated, as it checks the property 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


Re: [PR] MINIFICPP-2277 Add virtualenv support for python processors [nifi-minifi-cpp]

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


##########
extensions/python/PythonDependencyInstaller.cpp:
##########
@@ -0,0 +1,166 @@
+/**
+ *
+ * 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.
+ */
+#include "PythonDependencyInstaller.h"
+
+#include "PythonScriptException.h"
+#include "PythonInterpreter.h"
+#include "PyException.h"
+#include "types/Types.h"
+
+namespace org::apache::nifi::minifi::extensions::python {
+
+namespace {
+
+std::string getPythonBinary(const std::shared_ptr<Configure> &configuration) {
+#if WIN32
+  std::string python_binary_ = "python";
+#else
+  std::string python_binary_ = "python3";
+#endif
+  if (auto binary = configuration->get(minifi::Configuration::nifi_python_env_setup_binary)) {
+    python_binary_ = *binary;
+  }
+  return python_binary_;
+}
+
+// On Windows when calling a system command using std::system, the whole command needs to be encapsulated in additional quotes,
+// due to the std::system passing the command to 'cmd.exe /C' which needs the additional quotes to handle the command as a single argument
+std::string encapsulateCommandInQuotesIfNeeded(const std::string& command) {
+#if WIN32
+    return "\"" + command + "\"";
+#else
+    return command;
+#endif
+}
+
+}  // namespace
+
+PythonDependencyInstaller::PythonDependencyInstaller(const std::shared_ptr<Configure> &configuration) {
+  python_binary_ = getPythonBinary(configuration);
+  std::string automatic_install_str;
+  install_python_packages_automatically_ =
+    configuration->get(Configuration::nifi_python_install_packages_automatically, automatic_install_str) && utils::string::toBool(automatic_install_str).value_or(false);
+  if (auto path = configuration->get(minifi::Configuration::nifi_python_virtualenv_directory)) {
+    virtualenv_path_ = *path;
+    logger_->log_debug("Python virtualenv path was specified at: {}", virtualenv_path_.string());
+  } else {
+    logger_->log_debug("No valid python virtualenv path was specified");
+  }
+  if (auto python_processor_dir = configuration->get(minifi::Configuration::nifi_python_processor_dir)) {
+    python_processor_dir_ = *python_processor_dir;
+    logger_->log_debug("Python processor dir was specified at: {}", python_processor_dir_.string());
+  } else {
+    logger_->log_debug("No valid python processor dir was not specified in properties");
+  }
+  createVirtualEnvIfSpecified();
+  addVirtualenvToPath();
+}
+
+std::vector<std::filesystem::path> PythonDependencyInstaller::getRequirementsFilePaths() const {
+  if (!std::filesystem::exists(python_processor_dir_)) {
+    return {};
+  }
+  std::vector<std::filesystem::path> paths;
+  for (const auto& entry : std::filesystem::recursive_directory_iterator(std::filesystem::path{python_processor_dir_})) {
+    if (std::filesystem::is_regular_file(entry.path()) && entry.path().filename() == "requirements.txt") {
+      paths.push_back(entry.path());
+    }
+  }
+  return paths;
+}
+
+void PythonDependencyInstaller::createVirtualEnvIfSpecified() const {
+  if (virtualenv_path_.empty()) {

Review Comment:
   should we log a warning if the virtualenv_path is empty but the automatic install is requested?



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


Re: [PR] MINIFICPP-2277 Add virtualenv support for python processors [nifi-minifi-cpp]

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


##########
extensions/python/PythonConfigState.h:
##########
@@ -0,0 +1,50 @@
+/**
+ *
+ * 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.
+ */
+
+#pragma once
+
+#include <filesystem>
+#include <string>
+
+namespace org::apache::nifi::minifi::extensions::python {
+
+struct PythonConfigState {
+ public:
+  PythonConfigState(PythonConfigState&&) = delete;
+  PythonConfigState(const PythonConfigState&) = delete;
+  PythonConfigState& operator=(PythonConfigState&&) = delete;
+  PythonConfigState& operator=(const PythonConfigState&) = delete;
+
+  bool isPackageInstallationNeeded() const {
+    return install_python_packages_automatically && !virtualenv_path.empty();
+  }
+
+  static PythonConfigState& getInstance() {
+    static PythonConfigState config;
+    return config;
+  }

Review Comment:
   Since PythonScriptEngine is created on a per-processor basis, we can't let it handle the creation of the shared execution environment. That should be extracted to a separate class, which could be owned by e.g. PythonCreator, setting up the virtualenv and packages, so that individual PythonScriptEngines can be created in that environment. PythonConfigState could be part of this new class, and PythonCreator could inject the Configuration object into it. What do you think about this?



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


Re: [PR] MINIFICPP-2277 Add virtualenv support for python processors [nifi-minifi-cpp]

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


##########
extensions/python/PythonDependencyInstaller.cpp:
##########
@@ -0,0 +1,166 @@
+/**
+ *
+ * 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.
+ */
+#include "PythonDependencyInstaller.h"
+
+#include "PythonScriptException.h"
+#include "PythonInterpreter.h"
+#include "PyException.h"
+#include "types/Types.h"
+
+namespace org::apache::nifi::minifi::extensions::python {
+
+namespace {
+
+std::string getPythonBinary(const std::shared_ptr<Configure> &configuration) {
+#if WIN32
+  std::string python_binary_ = "python";
+#else
+  std::string python_binary_ = "python3";
+#endif
+  if (auto binary = configuration->get(minifi::Configuration::nifi_python_env_setup_binary)) {
+    python_binary_ = *binary;
+  }
+  return python_binary_;
+}
+
+// On Windows when calling a system command using std::system, the whole command needs to be encapsulated in additional quotes,
+// due to the std::system passing the command to 'cmd.exe /C' which needs the additional quotes to handle the command as a single argument
+std::string encapsulateCommandInQuotesIfNeeded(const std::string& command) {
+#if WIN32
+    return "\"" + command + "\"";
+#else
+    return command;
+#endif
+}
+
+}  // namespace
+
+PythonDependencyInstaller::PythonDependencyInstaller(const std::shared_ptr<Configure> &configuration) {
+  python_binary_ = getPythonBinary(configuration);
+  std::string automatic_install_str;
+  install_python_packages_automatically_ =
+    configuration->get(Configuration::nifi_python_install_packages_automatically, automatic_install_str) && utils::string::toBool(automatic_install_str).value_or(false);

Review Comment:
   we could use the monadic functions here



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


Re: [PR] MINIFICPP-2277 Add virtualenv support for python processors [nifi-minifi-cpp]

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


##########
extensions/python/PythonDependencyInstaller.h:
##########
@@ -0,0 +1,45 @@
+/**
+ *
+ * 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.
+ */
+#pragma once
+
+#include <memory>
+#include <filesystem>
+
+#include "PythonConfigState.h"
+#include "core/logging/Logger.h"
+#include "core/logging/LoggerConfiguration.h"
+#include "properties/Configure.h"
+
+namespace org::apache::nifi::minifi::extensions::python {
+
+class PythonDependencyInstaller {
+ public:
+  explicit PythonDependencyInstaller(const std::shared_ptr<Configure> &configuration);
+  void installDependenciesFromRequirementsFiles() const;
+
+ private:
+  std::vector<std::filesystem::path> getRequirementsFilePaths() const;
+  void createVirtualEnvIfSpecified() const;
+  static void evalScript(std::string_view script);
+  void addVirtualenvToPath() const;
+
+  PythonConfigState config_state_;

Review Comment:
   It may be worth moving the data from PythonConfigState to this class, for more simplicity. It doesn't really add a new responsibility, just simplifies the existing one.



##########
extensions/python/PythonDependencyInstaller.cpp:
##########
@@ -0,0 +1,167 @@
+/**
+ *
+ * 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.
+ */
+#include "PythonDependencyInstaller.h"
+
+#include "PythonScriptException.h"
+#include "PythonInterpreter.h"
+#include "PyException.h"
+#include "types/Types.h"
+
+namespace org::apache::nifi::minifi::extensions::python {
+
+namespace {
+
+std::string getPythonBinary(const std::shared_ptr<Configure> &configuration) {
+#if WIN32
+  std::string python_binary = "python";
+#else
+  std::string python_binary = "python3";
+#endif
+  if (auto binary = configuration->get(minifi::Configuration::nifi_python_env_setup_binary)) {
+    python_binary = *binary;
+  }
+  return python_binary;
+}
+
+// On Windows when calling a system command using std::system, the whole command needs to be encapsulated in additional quotes,
+// due to the std::system passing the command to 'cmd.exe /C' which needs the additional quotes to handle the command as a single argument
+std::string encapsulateCommandInQuotesIfNeeded(const std::string& command) {
+#if WIN32
+    return "\"" + command + "\"";
+#else
+    return command;
+#endif
+}
+
+}  // namespace
+
+PythonDependencyInstaller::PythonDependencyInstaller(const std::shared_ptr<Configure> &configuration) {
+  config_state_.python_binary = getPythonBinary(configuration);
+  std::string automatic_install_str;
+  config_state_.install_python_packages_automatically =
+    configuration->get(Configuration::nifi_python_install_packages_automatically, automatic_install_str) && utils::string::toBool(automatic_install_str).value_or(false);
+  if (auto path = configuration->get(minifi::Configuration::nifi_python_virtualenv_directory)) {
+    config_state_.virtualenv_path = *path;
+    logger_->log_debug("Python virtualenv path was specified at: {}", config_state_.virtualenv_path.string());
+  } else {
+    logger_->log_debug("No valid python virtualenv path was specified");
+  }
+  if (auto python_processor_dir = configuration->get(minifi::Configuration::nifi_python_processor_dir)) {
+    config_state_.python_processor_dir = *python_processor_dir;
+    logger_->log_debug("Python processor dir was specified at: {}", config_state_.python_processor_dir.string());
+  } else {
+    logger_->log_debug("No valid python processor dir was not specified in properties");
+  }
+  createVirtualEnvIfSpecified();
+  addVirtualenvToPath();
+}
+
+std::vector<std::filesystem::path> PythonDependencyInstaller::getRequirementsFilePaths() const {
+  if (!std::filesystem::exists(config_state_.python_processor_dir)) {
+    return {};
+  }
+  std::vector<std::filesystem::path> paths;
+  for (const auto& entry : std::filesystem::recursive_directory_iterator(std::filesystem::path{config_state_.python_processor_dir})) {
+    if (std::filesystem::is_regular_file(entry.path()) && entry.path().filename() == "requirements.txt") {
+      paths.push_back(entry.path());
+    }
+  }
+  return paths;
+}
+
+void PythonDependencyInstaller::createVirtualEnvIfSpecified() const {
+  if (config_state_.virtualenv_path.empty()) {
+    return;
+  }
+  if (!std::filesystem::exists(config_state_.virtualenv_path) || std::filesystem::is_empty(config_state_.virtualenv_path)) {
+    logger_->log_info("Creating python virtual env at: {}", config_state_.virtualenv_path.string());
+    auto venv_command = "\"" + config_state_.python_binary + "\" -m venv \"" + config_state_.virtualenv_path.string() + "\"";
+    auto return_value = std::system(encapsulateCommandInQuotesIfNeeded(venv_command).c_str());
+    if (return_value != 0) {
+      throw PythonScriptException(fmt::format("The following command creating python virtual env failed: '{}'", venv_command));
+    }
+  }
+}
+
+void PythonDependencyInstaller::installDependenciesFromRequirementsFiles() const {
+  std::string automatic_install_str;
+  if (!config_state_.isPackageInstallationNeeded()) {
+    return;
+  }
+  auto requirement_file_paths = getRequirementsFilePaths();
+  for (const auto& requirements_file_path : requirement_file_paths) {
+    logger_->log_info("Installing python packages from the following requirements.txt file: {}", requirements_file_path.string());
+    std::string pip_command;
+#if WIN32
+    pip_command.append("\"").append((config_state_.virtualenv_path / "Scripts" / "activate.bat").string()).append("\" && ");
+#else
+    pip_command.append(". \"").append((config_state_.virtualenv_path / "bin" / "activate").string()).append("\" && ");
+#endif
+    pip_command.append("\"").append(config_state_.python_binary).append("\" -m pip install --no-cache-dir -r \"").append(requirements_file_path.string()).append("\"");
+    auto return_value = std::system(encapsulateCommandInQuotesIfNeeded(pip_command).c_str());
+    if (return_value != 0) {
+      throw PythonScriptException(fmt::format("The following command to install python packages failed: '{}'", pip_command));
+    }
+  }
+}
+
+void PythonDependencyInstaller::evalScript(std::string_view script) {
+  GlobalInterpreterLock gil;
+  const auto script_file = "# -*- coding: utf-8 -*-\n" + std::string(script);

Review Comment:
   We could save a large string copy by using `utils::string::join_pack`. Even though the temporary string is an rvalue, the concatenated result can't reuse its buffer.



##########
extensions/python/PythonScriptEngine.h:
##########
@@ -84,7 +48,7 @@ class PythonScriptEngine {
   PythonScriptEngine& operator=(const PythonScriptEngine& other) = delete;
   PythonScriptEngine& operator=(PythonScriptEngine&& other) = delete;
 
-  static void initialize() {}
+  static void initialize(const std::shared_ptr<Configure> &configuration, const std::shared_ptr<core::logging::Logger>& logger);

Review Comment:
   Where is the implementation of this function?



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


Re: [PR] MINIFICPP-2277 Add virtualenv support for python processors [nifi-minifi-cpp]

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


##########
libminifi/src/Configuration.cpp:
##########
@@ -152,7 +152,11 @@ const std::unordered_map<std::string_view, gsl::not_null<const core::PropertyVal
   {Configuration::controller_socket_local_any_interface, gsl::make_not_null(&core::StandardPropertyTypes::BOOLEAN_TYPE)},
   {Configuration::controller_socket_host, gsl::make_not_null(&core::StandardPropertyTypes::VALID_TYPE)},
   {Configuration::controller_socket_port, gsl::make_not_null(&core::StandardPropertyTypes::PORT_TYPE)},
-  {Configuration::controller_ssl_context_service, gsl::make_not_null(&core::StandardPropertyTypes::VALID_TYPE)}
+  {Configuration::controller_ssl_context_service, gsl::make_not_null(&core::StandardPropertyTypes::VALID_TYPE)},
+  {Configuration::nifi_flow_file_repository_check_health, gsl::make_not_null(&core::StandardPropertyTypes::BOOLEAN_TYPE)},

Review Comment:
   Yes, the `nifi_flow_file_repository_check_health` was missing from the property map.



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


Re: [PR] MINIFICPP-2277 Add virtualenv support for python processors [nifi-minifi-cpp]

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


##########
extensions/python/PythonDependencyInstaller.cpp:
##########
@@ -0,0 +1,166 @@
+/**
+ *
+ * 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.
+ */
+#include "PythonDependencyInstaller.h"
+
+#include "PythonScriptException.h"
+#include "PythonInterpreter.h"
+#include "PyException.h"
+#include "types/Types.h"
+
+namespace org::apache::nifi::minifi::extensions::python {
+
+namespace {
+
+std::string getPythonBinary(const std::shared_ptr<Configure> &configuration) {
+#if WIN32
+  std::string python_binary_ = "python";
+#else
+  std::string python_binary_ = "python3";
+#endif
+  if (auto binary = configuration->get(minifi::Configuration::nifi_python_env_setup_binary)) {
+    python_binary_ = *binary;
+  }
+  return python_binary_;

Review Comment:
   accidental trailing underscore?



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


Re: [PR] MINIFICPP-2277 Add virtualenv support for python processors [nifi-minifi-cpp]

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


##########
docker/python-verify/conda.Dockerfile:
##########
@@ -31,14 +31,12 @@ USER root
 RUN wget https://repo.anaconda.com/archive/Anaconda3-2023.09-0-Linux-x86_64.sh -P /tmp \
     && echo "6c8a4abb36fbb711dc055b7049a23bbfd61d356de9468b41c5140f8a11abd851 /tmp/Anaconda3-2023.09-0-Linux-x86_64.sh" | sha256sum -c \
     && bash /tmp/Anaconda3-2023.09-0-Linux-x86_64.sh -b -p /opt/conda  \
-    && chown -R ${USER}:${USER} /opt/conda \
-    && mkdir /home/${USER}  \
-    && chown -R ${USER}:${USER} /home/${USER}
+    && chown -R ${USER}:${USER} /opt/conda
 
 USER ${USER}
 
 RUN ${CONDA_HOME}/bin/conda init bash
-RUN ${CONDA_HOME}/bin/conda install langchain -c conda-forge
+RUN ${CONDA_HOME}/bin/conda install "langchain<=0.17.0" -c conda-forge

Review Comment:
   Currently the latest version of langchain is 0.17.0 and as we run the python tests on multiple distributions and with virtualenvs or with conda in the CI, we run these tests with version 0.17.0 or bwloe. The tests with these versions are tested and passing, but the langchain library is rapidly changing so we don't want to use the latest version in the tests because they may break with a future version, so it is better to have the version fixed.



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


Re: [PR] MINIFICPP-2277 Add virtualenv support for python processors [nifi-minifi-cpp]

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


##########
extensions/python/PythonDependencyInstaller.cpp:
##########
@@ -0,0 +1,166 @@
+/**
+ *
+ * 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.
+ */
+#include "PythonDependencyInstaller.h"
+
+#include "PythonScriptException.h"
+#include "PythonInterpreter.h"
+#include "PyException.h"
+#include "types/Types.h"
+
+namespace org::apache::nifi::minifi::extensions::python {
+
+namespace {
+
+std::string getPythonBinary(const std::shared_ptr<Configure> &configuration) {
+#if WIN32
+  std::string python_binary_ = "python";
+#else
+  std::string python_binary_ = "python3";
+#endif
+  if (auto binary = configuration->get(minifi::Configuration::nifi_python_env_setup_binary)) {
+    python_binary_ = *binary;
+  }
+  return python_binary_;

Review Comment:
   Good catch, updated in 33e8770038e823678763c9bef60cc656c07ba16b



##########
extensions/python/PythonDependencyInstaller.cpp:
##########
@@ -0,0 +1,166 @@
+/**
+ *
+ * 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.
+ */
+#include "PythonDependencyInstaller.h"
+
+#include "PythonScriptException.h"
+#include "PythonInterpreter.h"
+#include "PyException.h"
+#include "types/Types.h"
+
+namespace org::apache::nifi::minifi::extensions::python {
+
+namespace {
+
+std::string getPythonBinary(const std::shared_ptr<Configure> &configuration) {
+#if WIN32
+  std::string python_binary_ = "python";
+#else
+  std::string python_binary_ = "python3";
+#endif
+  if (auto binary = configuration->get(minifi::Configuration::nifi_python_env_setup_binary)) {
+    python_binary_ = *binary;
+  }
+  return python_binary_;
+}
+
+// On Windows when calling a system command using std::system, the whole command needs to be encapsulated in additional quotes,
+// due to the std::system passing the command to 'cmd.exe /C' which needs the additional quotes to handle the command as a single argument
+std::string encapsulateCommandInQuotesIfNeeded(const std::string& command) {
+#if WIN32
+    return "\"" + command + "\"";
+#else
+    return command;
+#endif
+}
+
+}  // namespace
+
+PythonDependencyInstaller::PythonDependencyInstaller(const std::shared_ptr<Configure> &configuration) {
+  python_binary_ = getPythonBinary(configuration);
+  std::string automatic_install_str;
+  install_python_packages_automatically_ =
+    configuration->get(Configuration::nifi_python_install_packages_automatically, automatic_install_str) && utils::string::toBool(automatic_install_str).value_or(false);

Review Comment:
   Good idea, updated in 33e8770038e823678763c9bef60cc656c07ba16b



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


Re: [PR] MINIFICPP-2277 Add virtualenv support for python processors [nifi-minifi-cpp]

Posted by "szaszm (via GitHub)" <gi...@apache.org>.
szaszm closed pull request #1721: MINIFICPP-2277 Add virtualenv support for python processors
URL: https://github.com/apache/nifi-minifi-cpp/pull/1721


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


Re: [PR] MINIFICPP-2277 Add virtualenv support for python processors [nifi-minifi-cpp]

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


##########
extensions/python/PythonDependencyInstaller.h:
##########
@@ -0,0 +1,45 @@
+/**
+ *
+ * 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.
+ */
+#pragma once
+
+#include <memory>
+#include <filesystem>
+
+#include "PythonConfigState.h"
+#include "core/logging/Logger.h"
+#include "core/logging/LoggerConfiguration.h"
+#include "properties/Configure.h"
+
+namespace org::apache::nifi::minifi::extensions::python {
+
+class PythonDependencyInstaller {
+ public:
+  explicit PythonDependencyInstaller(const std::shared_ptr<Configure> &configuration);
+  void installDependenciesFromRequirementsFiles() const;
+
+ private:
+  std::vector<std::filesystem::path> getRequirementsFilePaths() const;
+  void createVirtualEnvIfSpecified() const;
+  static void evalScript(std::string_view script);
+  void addVirtualenvToPath() const;
+
+  PythonConfigState config_state_;

Review Comment:
   Updated in 99359a0eda994fd5efa3095ebb76811ff1136f05



##########
extensions/python/PythonDependencyInstaller.cpp:
##########
@@ -0,0 +1,167 @@
+/**
+ *
+ * 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.
+ */
+#include "PythonDependencyInstaller.h"
+
+#include "PythonScriptException.h"
+#include "PythonInterpreter.h"
+#include "PyException.h"
+#include "types/Types.h"
+
+namespace org::apache::nifi::minifi::extensions::python {
+
+namespace {
+
+std::string getPythonBinary(const std::shared_ptr<Configure> &configuration) {
+#if WIN32
+  std::string python_binary = "python";
+#else
+  std::string python_binary = "python3";
+#endif
+  if (auto binary = configuration->get(minifi::Configuration::nifi_python_env_setup_binary)) {
+    python_binary = *binary;
+  }
+  return python_binary;
+}
+
+// On Windows when calling a system command using std::system, the whole command needs to be encapsulated in additional quotes,
+// due to the std::system passing the command to 'cmd.exe /C' which needs the additional quotes to handle the command as a single argument
+std::string encapsulateCommandInQuotesIfNeeded(const std::string& command) {
+#if WIN32
+    return "\"" + command + "\"";
+#else
+    return command;
+#endif
+}
+
+}  // namespace
+
+PythonDependencyInstaller::PythonDependencyInstaller(const std::shared_ptr<Configure> &configuration) {
+  config_state_.python_binary = getPythonBinary(configuration);
+  std::string automatic_install_str;
+  config_state_.install_python_packages_automatically =
+    configuration->get(Configuration::nifi_python_install_packages_automatically, automatic_install_str) && utils::string::toBool(automatic_install_str).value_or(false);
+  if (auto path = configuration->get(minifi::Configuration::nifi_python_virtualenv_directory)) {
+    config_state_.virtualenv_path = *path;
+    logger_->log_debug("Python virtualenv path was specified at: {}", config_state_.virtualenv_path.string());
+  } else {
+    logger_->log_debug("No valid python virtualenv path was specified");
+  }
+  if (auto python_processor_dir = configuration->get(minifi::Configuration::nifi_python_processor_dir)) {
+    config_state_.python_processor_dir = *python_processor_dir;
+    logger_->log_debug("Python processor dir was specified at: {}", config_state_.python_processor_dir.string());
+  } else {
+    logger_->log_debug("No valid python processor dir was not specified in properties");
+  }
+  createVirtualEnvIfSpecified();
+  addVirtualenvToPath();
+}
+
+std::vector<std::filesystem::path> PythonDependencyInstaller::getRequirementsFilePaths() const {
+  if (!std::filesystem::exists(config_state_.python_processor_dir)) {
+    return {};
+  }
+  std::vector<std::filesystem::path> paths;
+  for (const auto& entry : std::filesystem::recursive_directory_iterator(std::filesystem::path{config_state_.python_processor_dir})) {
+    if (std::filesystem::is_regular_file(entry.path()) && entry.path().filename() == "requirements.txt") {
+      paths.push_back(entry.path());
+    }
+  }
+  return paths;
+}
+
+void PythonDependencyInstaller::createVirtualEnvIfSpecified() const {
+  if (config_state_.virtualenv_path.empty()) {
+    return;
+  }
+  if (!std::filesystem::exists(config_state_.virtualenv_path) || std::filesystem::is_empty(config_state_.virtualenv_path)) {
+    logger_->log_info("Creating python virtual env at: {}", config_state_.virtualenv_path.string());
+    auto venv_command = "\"" + config_state_.python_binary + "\" -m venv \"" + config_state_.virtualenv_path.string() + "\"";
+    auto return_value = std::system(encapsulateCommandInQuotesIfNeeded(venv_command).c_str());
+    if (return_value != 0) {
+      throw PythonScriptException(fmt::format("The following command creating python virtual env failed: '{}'", venv_command));
+    }
+  }
+}
+
+void PythonDependencyInstaller::installDependenciesFromRequirementsFiles() const {
+  std::string automatic_install_str;
+  if (!config_state_.isPackageInstallationNeeded()) {
+    return;
+  }
+  auto requirement_file_paths = getRequirementsFilePaths();
+  for (const auto& requirements_file_path : requirement_file_paths) {
+    logger_->log_info("Installing python packages from the following requirements.txt file: {}", requirements_file_path.string());
+    std::string pip_command;
+#if WIN32
+    pip_command.append("\"").append((config_state_.virtualenv_path / "Scripts" / "activate.bat").string()).append("\" && ");
+#else
+    pip_command.append(". \"").append((config_state_.virtualenv_path / "bin" / "activate").string()).append("\" && ");
+#endif
+    pip_command.append("\"").append(config_state_.python_binary).append("\" -m pip install --no-cache-dir -r \"").append(requirements_file_path.string()).append("\"");
+    auto return_value = std::system(encapsulateCommandInQuotesIfNeeded(pip_command).c_str());
+    if (return_value != 0) {
+      throw PythonScriptException(fmt::format("The following command to install python packages failed: '{}'", pip_command));
+    }
+  }
+}
+
+void PythonDependencyInstaller::evalScript(std::string_view script) {
+  GlobalInterpreterLock gil;
+  const auto script_file = "# -*- coding: utf-8 -*-\n" + std::string(script);

Review Comment:
   Good point, updated in 99359a0eda994fd5efa3095ebb76811ff1136f05



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


Re: [PR] MINIFICPP-2277 Add virtualenv support for python processors [nifi-minifi-cpp]

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


##########
docker/python-verify/conda.Dockerfile:
##########
@@ -31,14 +31,12 @@ USER root
 RUN wget https://repo.anaconda.com/archive/Anaconda3-2023.09-0-Linux-x86_64.sh -P /tmp \
     && echo "6c8a4abb36fbb711dc055b7049a23bbfd61d356de9468b41c5140f8a11abd851 /tmp/Anaconda3-2023.09-0-Linux-x86_64.sh" | sha256sum -c \
     && bash /tmp/Anaconda3-2023.09-0-Linux-x86_64.sh -b -p /opt/conda  \
-    && chown -R ${USER}:${USER} /opt/conda \
-    && mkdir /home/${USER}  \
-    && chown -R ${USER}:${USER} /home/${USER}
+    && chown -R ${USER}:${USER} /opt/conda
 
 USER ${USER}
 
 RUN ${CONDA_HOME}/bin/conda init bash
-RUN ${CONDA_HOME}/bin/conda install langchain -c conda-forge
+RUN ${CONDA_HOME}/bin/conda install "langchain<=0.17.0" -c conda-forge

Review Comment:
   What happened in langchain >0.17?



##########
extensions/python/PythonScriptEngine.cpp:
##########
@@ -68,6 +67,73 @@ void initThreads() {
 #pragma warning(pop)
 #endif
 }
+
+std::string encapsulateCommandInQuotesIfNeeded(const std::string& command) {
+#if WIN32
+    return "\"" + command + "\"";
+#else
+    return command;
+#endif
+}
+
+std::vector<std::filesystem::path> getRequirementsFilePaths(const std::shared_ptr<Configure> &configuration) {
+  std::vector<std::filesystem::path> paths;
+  if (auto python_processor_path = configuration->get(minifi::Configuration::nifi_python_processor_dir)) {
+    for (const auto& entry : std::filesystem::recursive_directory_iterator(std::filesystem::path{*python_processor_path})) {
+      if (std::filesystem::is_regular_file(entry.path()) && entry.path().filename() == "requirements.txt") {
+        paths.push_back(entry.path());
+      }
+    }
+  }
+  return paths;
+}
+
+std::string getPythonBinary(const std::shared_ptr<Configure> &configuration) {
+#if WIN32
+  std::string python_binary = "python";
+#else
+  std::string python_binary = "python3";
+#endif
+  if (auto binary = configuration->get(minifi::Configuration::nifi_python_env_setup_binary)) {
+    python_binary = *binary;
+  }
+  return python_binary;
+}
+
+void createVirtualEnvIfSpecified(const std::shared_ptr<Configure> &configuration) {
+  if (auto path = configuration->get(minifi::Configuration::nifi_python_virtualenv_directory)) {
+    PythonConfigState::getInstance().virtualenv_path = *path;
+    if (!std::filesystem::exists(PythonConfigState::getInstance().virtualenv_path) || !std::filesystem::is_empty(PythonConfigState::getInstance().virtualenv_path)) {
+      auto venv_command = "\"" + PythonConfigState::getInstance().python_binary + "\" -m venv \"" + PythonConfigState::getInstance().virtualenv_path.string() + "\"";
+      auto return_value = std::system(encapsulateCommandInQuotesIfNeeded(venv_command).c_str());
+      if (return_value != 0) {
+        throw PythonScriptException(fmt::format("The following command creating python virtual env failed: '{}'", venv_command));
+      }
+    }
+  }
+}
+
+void installPythonPackagesIfRequested(const std::shared_ptr<Configure> &configuration, const std::shared_ptr<core::logging::Logger>& logger) {
+  std::string automatic_install_str;
+  if (!PythonConfigState::getInstance().isPackageInstallationNeeded()) {
+    return;
+  }
+  auto requirement_file_paths = getRequirementsFilePaths(configuration);
+  for (const auto& requirements_file_path : requirement_file_paths) {
+    logger->log_info("Installing python packages from the following requirements.txt file: {}", requirements_file_path.string());
+    std::string pip_command;
+#if WIN32
+    pip_command.append("\"").append((PythonConfigState::getInstance().virtualenv_path / "Scripts" / "activate.bat").string()).append("\" && ");
+#else
+    pip_command.append(". \"").append((PythonConfigState::getInstance().virtualenv_path / "bin" / "activate").string()).append("\" && ");
+#endif
+    pip_command.append("\"").append(PythonConfigState::getInstance().python_binary).append("\" -m pip install --no-cache-dir -r \"").append(requirements_file_path.string()).append("\"");
+    auto return_value = std::system(encapsulateCommandInQuotesIfNeeded(pip_command).c_str());

Review Comment:
   Could you add this explanation into a code comment, maybe above the quoting function? I had the very same question when reading the code.



##########
encrypt-config/tests/ConfigFileEncryptorTests.cpp:
##########
@@ -77,7 +77,7 @@ TEST_CASE("ConfigFileEncryptor can encrypt the sensitive properties", "[encrypt-
     uint32_t num_properties_encrypted = encryptSensitivePropertiesInFile(test_file, KEY);
 
     REQUIRE(num_properties_encrypted == 1);
-    REQUIRE(test_file.size() == 110);
+    REQUIRE(test_file.size() == 115);

Review Comment:
   what's the increase?



##########
extensions/python/PythonConfigState.h:
##########
@@ -0,0 +1,50 @@
+/**
+ *
+ * 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.
+ */
+
+#pragma once
+
+#include <filesystem>
+#include <string>
+
+namespace org::apache::nifi::minifi::extensions::python {
+
+struct PythonConfigState {
+ public:
+  PythonConfigState(PythonConfigState&&) = delete;
+  PythonConfigState(const PythonConfigState&) = delete;
+  PythonConfigState& operator=(PythonConfigState&&) = delete;
+  PythonConfigState& operator=(const PythonConfigState&) = delete;
+
+  bool isPackageInstallationNeeded() const {
+    return install_python_packages_automatically && !virtualenv_path.empty();
+  }
+
+  static PythonConfigState& getInstance() {
+    static PythonConfigState config;
+    return config;
+  }

Review Comment:
   I'd prefer no new singletons. Would it be possible? Is there any reason for this being a singleton?



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


Re: [PR] MINIFICPP-2277 Add virtualenv support for python processors [nifi-minifi-cpp]

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


##########
extensions/python/PythonConfigState.h:
##########
@@ -0,0 +1,50 @@
+/**
+ *
+ * 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.
+ */
+
+#pragma once
+
+#include <filesystem>
+#include <string>
+
+namespace org::apache::nifi::minifi::extensions::python {
+
+struct PythonConfigState {
+ public:
+  PythonConfigState(PythonConfigState&&) = delete;
+  PythonConfigState(const PythonConfigState&) = delete;
+  PythonConfigState& operator=(PythonConfigState&&) = delete;
+  PythonConfigState& operator=(const PythonConfigState&) = delete;
+
+  bool isPackageInstallationNeeded() const {
+    return install_python_packages_automatically && !virtualenv_path.empty();
+  }
+
+  static PythonConfigState& getInstance() {
+    static PythonConfigState config;
+    return config;
+  }

Review Comment:
   Extracted dependency installation to the `PythonDependencyInstaller` class and moved the dependency install to `PythonCreator`, to install dependencies while processing all the python processor script files in 5b20de2ffdc4d8f2bfd8907d1a47294440a11fc2



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


Re: [PR] MINIFICPP-2277 Add virtualenv support for python processors [nifi-minifi-cpp]

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


##########
docker/python-verify/conda.Dockerfile:
##########
@@ -31,14 +31,12 @@ USER root
 RUN wget https://repo.anaconda.com/archive/Anaconda3-2023.09-0-Linux-x86_64.sh -P /tmp \
     && echo "6c8a4abb36fbb711dc055b7049a23bbfd61d356de9468b41c5140f8a11abd851 /tmp/Anaconda3-2023.09-0-Linux-x86_64.sh" | sha256sum -c \
     && bash /tmp/Anaconda3-2023.09-0-Linux-x86_64.sh -b -p /opt/conda  \
-    && chown -R ${USER}:${USER} /opt/conda \
-    && mkdir /home/${USER}  \
-    && chown -R ${USER}:${USER} /home/${USER}
+    && chown -R ${USER}:${USER} /opt/conda
 
 USER ${USER}
 
 RUN ${CONDA_HOME}/bin/conda init bash
-RUN ${CONDA_HOME}/bin/conda install langchain -c conda-forge
+RUN ${CONDA_HOME}/bin/conda install "langchain<=0.17.0" -c conda-forge

Review Comment:
   This is just for using fixed version of the library, to avoid breaking the tests with newer versions, see comment https://github.com/apache/nifi-minifi-cpp/pull/1721#discussion_r1513218719



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


Re: [PR] MINIFICPP-2277 Add virtualenv support for python processors [nifi-minifi-cpp]

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


##########
extensions/python/PYTHON.md:
##########
@@ -155,10 +158,33 @@ In the flow configuration these Python processors can be referenced by their ful
 
 Due to some differences between the NiFi and MiNiFi C++ processors and implementation, there are some limitations using the NiFi Python processors:
 - Record based processors are not yet supported in MiNiFi C++, so the NiFi Python processors inherited from RecordTransform are not supported.
-- Virtualenv support is not yet available in MiNiFi C++, so all required packages must be installed on the system.
 - Controller properties are not supported at the moment.
 - There are some validators in NiFi that are not present in MiNiFi C++, so some property validations will be missing using the NiFi Python processors.
 - Allowable values specified in NiFi Python processors are ignored in MiNiFi C++ (due to MiNiFi C++ requiring them to be specified at compile time), so the property values are not pre-verified.
 - MiNiFi C++ only supports expression language with flow file attributes, so only FLOWFILE_ATTRIBUTES expression language scope is supported, otherwise the expression language will not be evaluated.
 - MiNiFi C++ does not support property dependencies, so the property dependencies will be ignored. If a property depends on another property, the property will not be required.
 - MiNiFi C++ does not support the use of self.jvm member in Python processors that provides JVM bindings in NiFi, it is set to None in MiNiFi C++.
+- Inline definition of Python package dependencies, defined in the ProcessorDetails nested class are not supported as in NiFi, so the dependencies must be defined in the requirements.txt files. If a processor's dependencies are defined in the ProcessorDetails class, the dependencies should be copied to the requirements.txt file.
+
+## Use Python processors from virtualenv
+
+It is possible to set a virtualenv to be used by the Python processors in Apache MiNiFi C++. If the virtualenv directory is set, the Python processors will be executed using the packages installed in the virtualenv. If the virtualenv directory is not set, the Python processors will be executed using the packages installed on the system.
+
+    # in minifi.properties
+    nifi.python.virtualenv.directory=${MINIFI_HOME}/minifi-python-env
+
+**NOTE:* Using different python versions for the system and the virtualenv is not supported. The virtualenv must be created using the same python version as the system python.
+
+## Automatically install dependencies from requirements.txt files
+
+It is possible to automatically install the dependencies of the Python processors in the virtualenv defined in requirements.txt files. To enable this feature, the `nifi.python.install.packages.automatically` property must be set to true. If this property is set to true, then all requirements.txt files that appear under the MiNiFi Python directory and its subdirectories (defined by the `nifi.python.processor.dir` property) will be used to install the Python packages. If the `nifi.python.virtualenv.directory` property is set, the packages are installed in the virtualenv, otherwise this option is ignored. Due to install schema differences in different platforms, system level packages are expected to be installed manually by the user.

Review Comment:
   I find "... otherwise this option is ignored" unclear (which option is "this"?)  If it means that for automatic installation, both properties need to be set, I would write:
   
   ```suggestion
   It is possible to automatically install the dependencies of the Python processors defined in requirements.txt files into a virtualenv. To enable this feature, the `nifi.python.install.packages.automatically` property must be set to true, and the `nifi.python.virtualenv.directory` property must be set to a directory where a virtualenv either already exists, or it can be set up. In this case, all requirements.txt files that appear under the MiNiFi Python directory (defined by the `nifi.python.processor.dir` property) and its subdirectories will be used to install the Python packages into the given virtualenv. Due to install schema differences in different platforms, system level packages are expected to be installed manually by the user.
   ```



##########
extensions/python/PythonScriptEngine.cpp:
##########
@@ -68,6 +67,73 @@ void initThreads() {
 #pragma warning(pop)
 #endif
 }
+
+std::string encapsulateCommandInQuotesIfNeeded(const std::string& command) {
+#if WIN32
+    return "\"" + command + "\"";
+#else
+    return command;
+#endif
+}
+
+std::vector<std::filesystem::path> getRequirementsFilePaths(const std::shared_ptr<Configure> &configuration) {
+  std::vector<std::filesystem::path> paths;
+  if (auto python_processor_path = configuration->get(minifi::Configuration::nifi_python_processor_dir)) {
+    for (const auto& entry : std::filesystem::recursive_directory_iterator(std::filesystem::path{*python_processor_path})) {
+      if (std::filesystem::is_regular_file(entry.path()) && entry.path().filename() == "requirements.txt") {
+        paths.push_back(entry.path());
+      }
+    }
+  }
+  return paths;
+}
+
+std::string getPythonBinary(const std::shared_ptr<Configure> &configuration) {
+#if WIN32
+  std::string python_binary = "python";
+#else
+  std::string python_binary = "python3";
+#endif
+  if (auto binary = configuration->get(minifi::Configuration::nifi_python_env_setup_binary)) {
+    python_binary = *binary;
+  }
+  return python_binary;
+}
+
+void createVirtualEnvIfSpecified(const std::shared_ptr<Configure> &configuration) {
+  if (auto path = configuration->get(minifi::Configuration::nifi_python_virtualenv_directory)) {
+    PythonConfigState::getInstance().virtualenv_path = *path;
+    if (!std::filesystem::exists(PythonConfigState::getInstance().virtualenv_path) || !std::filesystem::is_empty(PythonConfigState::getInstance().virtualenv_path)) {

Review Comment:
   why is the virtualenv creation skipped if the directory is empty?



##########
extensions/python/PythonConfigState.h:
##########
@@ -0,0 +1,67 @@
+/**
+ *
+ * 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.
+ */
+
+#pragma once
+
+#include <vector>
+#include <utility>
+#include <algorithm>
+#include <string>
+#include <memory>
+#include <filesystem>
+#include "core/Core.h"
+#include "core/logging/LoggerConfiguration.h"
+#include "core/Resource.h"
+#include "ExecutePythonProcessor.h"
+#include "PythonObjectFactory.h"
+#include "agent/agent_version.h"
+#include "agent/build_description.h"
+#include "utils/file/FileUtils.h"
+#include "utils/StringUtils.h"
+#include "range/v3/algorithm.hpp"
+#include "properties/Configuration.h"
+#include "utils/file/FilePattern.h"
+#include "range/v3/view/filter.hpp"

Review Comment:
   are all these `#include`s needed in this header file?



##########
docker/test/integration/cluster/containers/MinifiContainer.py:
##########
@@ -33,7 +33,11 @@ def __init__(self):
         self.enable_prometheus = False
         self.enable_prometheus_with_ssl = False
         self.enable_sql = False
-        self.use_nifi_python_processors = False
+        self.use_nifi_python_processors_with_system_python_packages_installed = False
+        self.use_nifi_python_processors_with_virtualenv = False
+        self.use_nifi_python_processors_with_virtualenv_packages_installed = False
+        self.create_python_virtualenv = False
+        self.install = False

Review Comment:
   Where are these two used?



##########
docker/python-verify/conda.Dockerfile:
##########
@@ -31,14 +31,12 @@ USER root
 RUN wget https://repo.anaconda.com/archive/Anaconda3-2023.09-0-Linux-x86_64.sh -P /tmp \
     && echo "6c8a4abb36fbb711dc055b7049a23bbfd61d356de9468b41c5140f8a11abd851 /tmp/Anaconda3-2023.09-0-Linux-x86_64.sh" | sha256sum -c \
     && bash /tmp/Anaconda3-2023.09-0-Linux-x86_64.sh -b -p /opt/conda  \
-    && chown -R ${USER}:${USER} /opt/conda \
-    && mkdir /home/${USER}  \
-    && chown -R ${USER}:${USER} /home/${USER}
+    && chown -R ${USER}:${USER} /opt/conda
 
 USER ${USER}
 
 RUN ${CONDA_HOME}/bin/conda init bash
-RUN ${CONDA_HOME}/bin/conda install langchain -c conda-forge
+RUN ${CONDA_HOME}/bin/conda install "langchain<=0.17.0" -c conda-forge

Review Comment:
   Why `<= 0.17.0`? A comment could be useful so we can remove this restriction when it is no longer needed.



##########
extensions/python/PythonScriptEngine.cpp:
##########
@@ -68,6 +67,73 @@ void initThreads() {
 #pragma warning(pop)
 #endif
 }
+
+std::string encapsulateCommandInQuotesIfNeeded(const std::string& command) {
+#if WIN32
+    return "\"" + command + "\"";
+#else
+    return command;
+#endif
+}
+
+std::vector<std::filesystem::path> getRequirementsFilePaths(const std::shared_ptr<Configure> &configuration) {
+  std::vector<std::filesystem::path> paths;
+  if (auto python_processor_path = configuration->get(minifi::Configuration::nifi_python_processor_dir)) {
+    for (const auto& entry : std::filesystem::recursive_directory_iterator(std::filesystem::path{*python_processor_path})) {
+      if (std::filesystem::is_regular_file(entry.path()) && entry.path().filename() == "requirements.txt") {
+        paths.push_back(entry.path());
+      }
+    }
+  }
+  return paths;
+}
+
+std::string getPythonBinary(const std::shared_ptr<Configure> &configuration) {
+#if WIN32
+  std::string python_binary = "python";
+#else
+  std::string python_binary = "python3";
+#endif
+  if (auto binary = configuration->get(minifi::Configuration::nifi_python_env_setup_binary)) {
+    python_binary = *binary;
+  }
+  return python_binary;
+}
+
+void createVirtualEnvIfSpecified(const std::shared_ptr<Configure> &configuration) {
+  if (auto path = configuration->get(minifi::Configuration::nifi_python_virtualenv_directory)) {
+    PythonConfigState::getInstance().virtualenv_path = *path;
+    if (!std::filesystem::exists(PythonConfigState::getInstance().virtualenv_path) || !std::filesystem::is_empty(PythonConfigState::getInstance().virtualenv_path)) {
+      auto venv_command = "\"" + PythonConfigState::getInstance().python_binary + "\" -m venv \"" + PythonConfigState::getInstance().virtualenv_path.string() + "\"";
+      auto return_value = std::system(encapsulateCommandInQuotesIfNeeded(venv_command).c_str());
+      if (return_value != 0) {
+        throw PythonScriptException(fmt::format("The following command creating python virtual env failed: '{}'", venv_command));
+      }
+    }
+  }
+}
+
+void installPythonPackagesIfRequested(const std::shared_ptr<Configure> &configuration, const std::shared_ptr<core::logging::Logger>& logger) {
+  std::string automatic_install_str;
+  if (!PythonConfigState::getInstance().isPackageInstallationNeeded()) {
+    return;
+  }
+  auto requirement_file_paths = getRequirementsFilePaths(configuration);
+  for (const auto& requirements_file_path : requirement_file_paths) {
+    logger->log_info("Installing python packages from the following requirements.txt file: {}", requirements_file_path.string());
+    std::string pip_command;
+#if WIN32
+    pip_command.append("\"").append((PythonConfigState::getInstance().virtualenv_path / "Scripts" / "activate.bat").string()).append("\" && ");
+#else
+    pip_command.append(". \"").append((PythonConfigState::getInstance().virtualenv_path / "bin" / "activate").string()).append("\" && ");
+#endif
+    pip_command.append("\"").append(PythonConfigState::getInstance().python_binary).append("\" -m pip install --no-cache-dir -r \"").append(requirements_file_path.string()).append("\"");
+    auto return_value = std::system(encapsulateCommandInQuotesIfNeeded(pip_command).c_str());

Review Comment:
   Do we need `encapsulateCommandInQuotesIfNeeded`?  There are already quotes around the executable path-and-filename above.  In fact, is it going to work?  We'll end up with something like `""\venv_path\Scripts\activate.bat" && "python" -m pip install --no-cache-dir -r "\requirements_path\requirements.txt""`, ie. double quotes at the beginning and the end, which looks strange.



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


Re: [PR] MINIFICPP-2277 Add virtualenv support for python processors [nifi-minifi-cpp]

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


##########
extensions/python/PythonScriptEngine.cpp:
##########
@@ -68,6 +67,73 @@ void initThreads() {
 #pragma warning(pop)
 #endif
 }
+
+std::string encapsulateCommandInQuotesIfNeeded(const std::string& command) {
+#if WIN32
+    return "\"" + command + "\"";
+#else
+    return command;
+#endif
+}
+
+std::vector<std::filesystem::path> getRequirementsFilePaths(const std::shared_ptr<Configure> &configuration) {
+  std::vector<std::filesystem::path> paths;
+  if (auto python_processor_path = configuration->get(minifi::Configuration::nifi_python_processor_dir)) {
+    for (const auto& entry : std::filesystem::recursive_directory_iterator(std::filesystem::path{*python_processor_path})) {
+      if (std::filesystem::is_regular_file(entry.path()) && entry.path().filename() == "requirements.txt") {
+        paths.push_back(entry.path());
+      }
+    }
+  }
+  return paths;
+}
+
+std::string getPythonBinary(const std::shared_ptr<Configure> &configuration) {
+#if WIN32
+  std::string python_binary = "python";
+#else
+  std::string python_binary = "python3";
+#endif
+  if (auto binary = configuration->get(minifi::Configuration::nifi_python_env_setup_binary)) {
+    python_binary = *binary;
+  }
+  return python_binary;
+}
+
+void createVirtualEnvIfSpecified(const std::shared_ptr<Configure> &configuration) {
+  if (auto path = configuration->get(minifi::Configuration::nifi_python_virtualenv_directory)) {
+    PythonConfigState::getInstance().virtualenv_path = *path;
+    if (!std::filesystem::exists(PythonConfigState::getInstance().virtualenv_path) || !std::filesystem::is_empty(PythonConfigState::getInstance().virtualenv_path)) {

Review Comment:
   Good catch, we should only create a virtualenv if the directory is empty, probably didn't update the condition while refactoring and didn't retest this, updated in 44654b11b315ed6010424cb72abb334b8b286377



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


Re: [PR] MINIFICPP-2277 Add virtualenv support for python processors [nifi-minifi-cpp]

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


##########
docker/test/integration/cluster/containers/MinifiContainer.py:
##########
@@ -33,7 +33,11 @@ def __init__(self):
         self.enable_prometheus = False
         self.enable_prometheus_with_ssl = False
         self.enable_sql = False
-        self.use_nifi_python_processors = False
+        self.use_nifi_python_processors_with_system_python_packages_installed = False
+        self.use_nifi_python_processors_with_virtualenv = False
+        self.use_nifi_python_processors_with_virtualenv_packages_installed = False
+        self.create_python_virtualenv = False
+        self.install = False

Review Comment:
   They are unused, removed in 44654b11b315ed6010424cb72abb334b8b286377



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


Re: [PR] MINIFICPP-2277 Add virtualenv support for python processors [nifi-minifi-cpp]

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


##########
extensions/python/PythonConfigState.h:
##########
@@ -0,0 +1,50 @@
+/**
+ *
+ * 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.
+ */
+
+#pragma once
+
+#include <filesystem>
+#include <string>
+
+namespace org::apache::nifi::minifi::extensions::python {
+
+struct PythonConfigState {
+ public:
+  PythonConfigState(PythonConfigState&&) = delete;
+  PythonConfigState(const PythonConfigState&) = delete;
+  PythonConfigState& operator=(PythonConfigState&&) = delete;
+  PythonConfigState& operator=(const PythonConfigState&) = delete;
+
+  bool isPackageInstallationNeeded() const {
+    return install_python_packages_automatically && !virtualenv_path.empty();
+  }
+
+  static PythonConfigState& getInstance() {
+    static PythonConfigState config;
+    return config;
+  }

Review Comment:
   You are right, I also try to avoid them when possible. In this case the python configuration is read in the PythonCreator from the Configuration object, but has to be used in the PythonScriptEngine which is initialized in each python processor. Due to the interface we use to create the python processors (instantiating the ExecutePythonProcessor class from the PythonObjectFactory) there is no clean way currently to reach the `Configuration` object from the processors' scope. Even though it's better to avoid them I think this is still a cleaner solution than anything else I could come up with.



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


Re: [PR] MINIFICPP-2277 Add virtualenv support for python processors [nifi-minifi-cpp]

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


##########
extensions/python/PythonScriptEngine.h:
##########
@@ -84,7 +48,7 @@ class PythonScriptEngine {
   PythonScriptEngine& operator=(const PythonScriptEngine& other) = delete;
   PythonScriptEngine& operator=(PythonScriptEngine&& other) = delete;
 
-  static void initialize() {}
+  static void initialize(const std::shared_ptr<Configure> &configuration, const std::shared_ptr<core::logging::Logger>& logger);

Review Comment:
   Good catch, it was removed, but the declaration remained, removed in 99359a0eda994fd5efa3095ebb76811ff1136f05



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


Re: [PR] MINIFICPP-2277 Add virtualenv support for python processors [nifi-minifi-cpp]

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


##########
extensions/python/ExecutePythonProcessor.h:
##########
@@ -35,10 +35,6 @@
 #include "PythonScriptEngine.h"
 #include "PythonScriptEngine.h"
 
-#if defined(__GNUC__) || defined(__GNUG__)
-#pragma GCC visibility push(hidden)

Review Comment:
   This was added due to pybind11 requiring -fvisibility=hidden, but as we do not use pybind11 anymore this can be removed



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


Re: [PR] MINIFICPP-2277 Add virtualenv support for python processors [nifi-minifi-cpp]

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


##########
extensions/python/PythonScriptEngine.cpp:
##########
@@ -68,6 +67,73 @@ void initThreads() {
 #pragma warning(pop)
 #endif
 }
+
+std::string encapsulateCommandInQuotesIfNeeded(const std::string& command) {
+#if WIN32
+    return "\"" + command + "\"";
+#else
+    return command;
+#endif
+}
+
+std::vector<std::filesystem::path> getRequirementsFilePaths(const std::shared_ptr<Configure> &configuration) {
+  std::vector<std::filesystem::path> paths;
+  if (auto python_processor_path = configuration->get(minifi::Configuration::nifi_python_processor_dir)) {
+    for (const auto& entry : std::filesystem::recursive_directory_iterator(std::filesystem::path{*python_processor_path})) {
+      if (std::filesystem::is_regular_file(entry.path()) && entry.path().filename() == "requirements.txt") {
+        paths.push_back(entry.path());
+      }
+    }
+  }
+  return paths;
+}
+
+std::string getPythonBinary(const std::shared_ptr<Configure> &configuration) {
+#if WIN32
+  std::string python_binary = "python";
+#else
+  std::string python_binary = "python3";
+#endif
+  if (auto binary = configuration->get(minifi::Configuration::nifi_python_env_setup_binary)) {
+    python_binary = *binary;
+  }
+  return python_binary;
+}
+
+void createVirtualEnvIfSpecified(const std::shared_ptr<Configure> &configuration) {
+  if (auto path = configuration->get(minifi::Configuration::nifi_python_virtualenv_directory)) {
+    PythonConfigState::getInstance().virtualenv_path = *path;
+    if (!std::filesystem::exists(PythonConfigState::getInstance().virtualenv_path) || !std::filesystem::is_empty(PythonConfigState::getInstance().virtualenv_path)) {
+      auto venv_command = "\"" + PythonConfigState::getInstance().python_binary + "\" -m venv \"" + PythonConfigState::getInstance().virtualenv_path.string() + "\"";
+      auto return_value = std::system(encapsulateCommandInQuotesIfNeeded(venv_command).c_str());
+      if (return_value != 0) {
+        throw PythonScriptException(fmt::format("The following command creating python virtual env failed: '{}'", venv_command));
+      }
+    }
+  }
+}
+
+void installPythonPackagesIfRequested(const std::shared_ptr<Configure> &configuration, const std::shared_ptr<core::logging::Logger>& logger) {
+  std::string automatic_install_str;
+  if (!PythonConfigState::getInstance().isPackageInstallationNeeded()) {
+    return;
+  }
+  auto requirement_file_paths = getRequirementsFilePaths(configuration);
+  for (const auto& requirements_file_path : requirement_file_paths) {
+    logger->log_info("Installing python packages from the following requirements.txt file: {}", requirements_file_path.string());
+    std::string pip_command;
+#if WIN32
+    pip_command.append("\"").append((PythonConfigState::getInstance().virtualenv_path / "Scripts" / "activate.bat").string()).append("\" && ");
+#else
+    pip_command.append(". \"").append((PythonConfigState::getInstance().virtualenv_path / "bin" / "activate").string()).append("\" && ");
+#endif
+    pip_command.append("\"").append(PythonConfigState::getInstance().python_binary).append("\" -m pip install --no-cache-dir -r \"").append(requirements_file_path.string()).append("\"");
+    auto return_value = std::system(encapsulateCommandInQuotesIfNeeded(pip_command).c_str());

Review Comment:
   Yes it looks strange, but there is an issue with std::system on Windows that needs the additional quotes on the whole command to work correctly. If I remember well it passes the system command as a parameter to the `cmd.exe /C` command and the additional quotes are needed to parse the system command's arguments correctly.



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


Re: [PR] MINIFICPP-2277 Add virtualenv support for python processors [nifi-minifi-cpp]

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


##########
libminifi/src/Configuration.cpp:
##########
@@ -152,7 +152,11 @@ const std::unordered_map<std::string_view, gsl::not_null<const core::PropertyVal
   {Configuration::controller_socket_local_any_interface, gsl::make_not_null(&core::StandardPropertyTypes::BOOLEAN_TYPE)},
   {Configuration::controller_socket_host, gsl::make_not_null(&core::StandardPropertyTypes::VALID_TYPE)},
   {Configuration::controller_socket_port, gsl::make_not_null(&core::StandardPropertyTypes::PORT_TYPE)},
-  {Configuration::controller_ssl_context_service, gsl::make_not_null(&core::StandardPropertyTypes::VALID_TYPE)}
+  {Configuration::controller_ssl_context_service, gsl::make_not_null(&core::StandardPropertyTypes::VALID_TYPE)},
+  {Configuration::nifi_flow_file_repository_check_health, gsl::make_not_null(&core::StandardPropertyTypes::BOOLEAN_TYPE)},

Review Comment:
   This line was missing?



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


Re: [PR] MINIFICPP-2277 Add virtualenv support for python processors [nifi-minifi-cpp]

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


##########
extensions/python/PythonScriptEngine.cpp:
##########
@@ -68,6 +67,73 @@ void initThreads() {
 #pragma warning(pop)
 #endif
 }
+
+std::string encapsulateCommandInQuotesIfNeeded(const std::string& command) {
+#if WIN32
+    return "\"" + command + "\"";
+#else
+    return command;
+#endif
+}
+
+std::vector<std::filesystem::path> getRequirementsFilePaths(const std::shared_ptr<Configure> &configuration) {
+  std::vector<std::filesystem::path> paths;
+  if (auto python_processor_path = configuration->get(minifi::Configuration::nifi_python_processor_dir)) {
+    for (const auto& entry : std::filesystem::recursive_directory_iterator(std::filesystem::path{*python_processor_path})) {
+      if (std::filesystem::is_regular_file(entry.path()) && entry.path().filename() == "requirements.txt") {
+        paths.push_back(entry.path());
+      }
+    }
+  }
+  return paths;
+}
+
+std::string getPythonBinary(const std::shared_ptr<Configure> &configuration) {
+#if WIN32
+  std::string python_binary = "python";
+#else
+  std::string python_binary = "python3";
+#endif
+  if (auto binary = configuration->get(minifi::Configuration::nifi_python_env_setup_binary)) {
+    python_binary = *binary;
+  }
+  return python_binary;
+}
+
+void createVirtualEnvIfSpecified(const std::shared_ptr<Configure> &configuration) {
+  if (auto path = configuration->get(minifi::Configuration::nifi_python_virtualenv_directory)) {
+    PythonConfigState::getInstance().virtualenv_path = *path;
+    if (!std::filesystem::exists(PythonConfigState::getInstance().virtualenv_path) || !std::filesystem::is_empty(PythonConfigState::getInstance().virtualenv_path)) {
+      auto venv_command = "\"" + PythonConfigState::getInstance().python_binary + "\" -m venv \"" + PythonConfigState::getInstance().virtualenv_path.string() + "\"";
+      auto return_value = std::system(encapsulateCommandInQuotesIfNeeded(venv_command).c_str());
+      if (return_value != 0) {
+        throw PythonScriptException(fmt::format("The following command creating python virtual env failed: '{}'", venv_command));
+      }
+    }
+  }
+}
+
+void installPythonPackagesIfRequested(const std::shared_ptr<Configure> &configuration, const std::shared_ptr<core::logging::Logger>& logger) {
+  std::string automatic_install_str;
+  if (!PythonConfigState::getInstance().isPackageInstallationNeeded()) {
+    return;
+  }
+  auto requirement_file_paths = getRequirementsFilePaths(configuration);
+  for (const auto& requirements_file_path : requirement_file_paths) {
+    logger->log_info("Installing python packages from the following requirements.txt file: {}", requirements_file_path.string());
+    std::string pip_command;
+#if WIN32
+    pip_command.append("\"").append((PythonConfigState::getInstance().virtualenv_path / "Scripts" / "activate.bat").string()).append("\" && ");
+#else
+    pip_command.append(". \"").append((PythonConfigState::getInstance().virtualenv_path / "bin" / "activate").string()).append("\" && ");
+#endif
+    pip_command.append("\"").append(PythonConfigState::getInstance().python_binary).append("\" -m pip install --no-cache-dir -r \"").append(requirements_file_path.string()).append("\"");
+    auto return_value = std::system(encapsulateCommandInQuotesIfNeeded(pip_command).c_str());

Review Comment:
   Added comment in c8e483279209a6d6a843e0f343876ff9adb97c93



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


Re: [PR] MINIFICPP-2277 Add virtualenv support for python processors [nifi-minifi-cpp]

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


##########
encrypt-config/tests/ConfigFileEncryptorTests.cpp:
##########
@@ -77,7 +77,7 @@ TEST_CASE("ConfigFileEncryptor can encrypt the sensitive properties", "[encrypt-
     uint32_t num_properties_encrypted = encryptSensitivePropertiesInFile(test_file, KEY);
 
     REQUIRE(num_properties_encrypted == 1);
-    REQUIRE(test_file.size() == 110);
+    REQUIRE(test_file.size() == 115);

Review Comment:
   When the default minifi.properties file changes this tests always needs to be updated, as it checks the line 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


Re: [PR] MINIFICPP-2277 Add virtualenv support for python processors [nifi-minifi-cpp]

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


##########
extensions/python/PythonDependencyInstaller.cpp:
##########
@@ -0,0 +1,166 @@
+/**
+ *
+ * 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.
+ */
+#include "PythonDependencyInstaller.h"
+
+#include "PythonScriptException.h"
+#include "PythonInterpreter.h"
+#include "PyException.h"
+#include "types/Types.h"
+
+namespace org::apache::nifi::minifi::extensions::python {
+
+namespace {
+
+std::string getPythonBinary(const std::shared_ptr<Configure> &configuration) {
+#if WIN32
+  std::string python_binary_ = "python";
+#else
+  std::string python_binary_ = "python3";
+#endif
+  if (auto binary = configuration->get(minifi::Configuration::nifi_python_env_setup_binary)) {
+    python_binary_ = *binary;
+  }
+  return python_binary_;
+}
+
+// On Windows when calling a system command using std::system, the whole command needs to be encapsulated in additional quotes,
+// due to the std::system passing the command to 'cmd.exe /C' which needs the additional quotes to handle the command as a single argument
+std::string encapsulateCommandInQuotesIfNeeded(const std::string& command) {
+#if WIN32
+    return "\"" + command + "\"";
+#else
+    return command;
+#endif
+}
+
+}  // namespace
+
+PythonDependencyInstaller::PythonDependencyInstaller(const std::shared_ptr<Configure> &configuration) {
+  python_binary_ = getPythonBinary(configuration);
+  std::string automatic_install_str;
+  install_python_packages_automatically_ =
+    configuration->get(Configuration::nifi_python_install_packages_automatically, automatic_install_str) && utils::string::toBool(automatic_install_str).value_or(false);
+  if (auto path = configuration->get(minifi::Configuration::nifi_python_virtualenv_directory)) {
+    virtualenv_path_ = *path;
+    logger_->log_debug("Python virtualenv path was specified at: {}", virtualenv_path_.string());
+  } else {
+    logger_->log_debug("No valid python virtualenv path was specified");
+  }
+  if (auto python_processor_dir = configuration->get(minifi::Configuration::nifi_python_processor_dir)) {
+    python_processor_dir_ = *python_processor_dir;
+    logger_->log_debug("Python processor dir was specified at: {}", python_processor_dir_.string());
+  } else {
+    logger_->log_debug("No valid python processor dir was not specified in properties");
+  }
+  createVirtualEnvIfSpecified();
+  addVirtualenvToPath();
+}
+
+std::vector<std::filesystem::path> PythonDependencyInstaller::getRequirementsFilePaths() const {
+  if (!std::filesystem::exists(python_processor_dir_)) {
+    return {};
+  }
+  std::vector<std::filesystem::path> paths;
+  for (const auto& entry : std::filesystem::recursive_directory_iterator(std::filesystem::path{python_processor_dir_})) {
+    if (std::filesystem::is_regular_file(entry.path()) && entry.path().filename() == "requirements.txt") {
+      paths.push_back(entry.path());
+    }
+  }
+  return paths;
+}
+
+void PythonDependencyInstaller::createVirtualEnvIfSpecified() const {
+  if (virtualenv_path_.empty()) {

Review Comment:
   Sounds good, added warning in 33e8770038e823678763c9bef60cc656c07ba16b



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


Re: [PR] MINIFICPP-2277 Add virtualenv support for python processors [nifi-minifi-cpp]

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


##########
encrypt-config/tests/ConfigFileEncryptorTests.cpp:
##########
@@ -77,7 +77,7 @@ TEST_CASE("ConfigFileEncryptor can encrypt the sensitive properties", "[encrypt-
     uint32_t num_properties_encrypted = encryptSensitivePropertiesInFile(test_file, KEY);
 
     REQUIRE(num_properties_encrypted == 1);
-    REQUIRE(test_file.size() == 110);
+    REQUIRE(test_file.size() == 115);

Review Comment:
   When the default minifi.properties file changes this test always needs to be updated, as it checks the line 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


Re: [PR] MINIFICPP-2277 Add virtualenv support for python processors [nifi-minifi-cpp]

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


##########
extensions/python/PYTHON.md:
##########
@@ -155,10 +158,33 @@ In the flow configuration these Python processors can be referenced by their ful
 
 Due to some differences between the NiFi and MiNiFi C++ processors and implementation, there are some limitations using the NiFi Python processors:
 - Record based processors are not yet supported in MiNiFi C++, so the NiFi Python processors inherited from RecordTransform are not supported.
-- Virtualenv support is not yet available in MiNiFi C++, so all required packages must be installed on the system.
 - Controller properties are not supported at the moment.
 - There are some validators in NiFi that are not present in MiNiFi C++, so some property validations will be missing using the NiFi Python processors.
 - Allowable values specified in NiFi Python processors are ignored in MiNiFi C++ (due to MiNiFi C++ requiring them to be specified at compile time), so the property values are not pre-verified.
 - MiNiFi C++ only supports expression language with flow file attributes, so only FLOWFILE_ATTRIBUTES expression language scope is supported, otherwise the expression language will not be evaluated.
 - MiNiFi C++ does not support property dependencies, so the property dependencies will be ignored. If a property depends on another property, the property will not be required.
 - MiNiFi C++ does not support the use of self.jvm member in Python processors that provides JVM bindings in NiFi, it is set to None in MiNiFi C++.
+- Inline definition of Python package dependencies, defined in the ProcessorDetails nested class are not supported as in NiFi, so the dependencies must be defined in the requirements.txt files. If a processor's dependencies are defined in the ProcessorDetails class, the dependencies should be copied to the requirements.txt file.
+
+## Use Python processors from virtualenv
+
+It is possible to set a virtualenv to be used by the Python processors in Apache MiNiFi C++. If the virtualenv directory is set, the Python processors will be executed using the packages installed in the virtualenv. If the virtualenv directory is not set, the Python processors will be executed using the packages installed on the system.
+
+    # in minifi.properties
+    nifi.python.virtualenv.directory=${MINIFI_HOME}/minifi-python-env
+
+**NOTE:* Using different python versions for the system and the virtualenv is not supported. The virtualenv must be created using the same python version as the system python.
+
+## Automatically install dependencies from requirements.txt files
+
+It is possible to automatically install the dependencies of the Python processors in the virtualenv defined in requirements.txt files. To enable this feature, the `nifi.python.install.packages.automatically` property must be set to true. If this property is set to true, then all requirements.txt files that appear under the MiNiFi Python directory and its subdirectories (defined by the `nifi.python.processor.dir` property) will be used to install the Python packages. If the `nifi.python.virtualenv.directory` property is set, the packages are installed in the virtualenv, otherwise this option is ignored. Due to install schema differences in different platforms, system level packages are expected to be installed manually by the user.

Review Comment:
   You are right it's more clear this way, updated in 44654b11b315ed6010424cb72abb334b8b286377



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


Re: [PR] MINIFICPP-2277 Add virtualenv support for python processors [nifi-minifi-cpp]

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


##########
extensions/python/PythonConfigState.h:
##########
@@ -0,0 +1,67 @@
+/**
+ *
+ * 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.
+ */
+
+#pragma once
+
+#include <vector>
+#include <utility>
+#include <algorithm>
+#include <string>
+#include <memory>
+#include <filesystem>
+#include "core/Core.h"
+#include "core/logging/LoggerConfiguration.h"
+#include "core/Resource.h"
+#include "ExecutePythonProcessor.h"
+#include "PythonObjectFactory.h"
+#include "agent/agent_version.h"
+#include "agent/build_description.h"
+#include "utils/file/FileUtils.h"
+#include "utils/StringUtils.h"
+#include "range/v3/algorithm.hpp"
+#include "properties/Configuration.h"
+#include "utils/file/FilePattern.h"
+#include "range/v3/view/filter.hpp"

Review Comment:
   Good point, left them in by mistake, removed the unneeded ones in 44654b11b315ed6010424cb72abb334b8b286377



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