You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by GitBox <gi...@apache.org> on 2022/12/01 19:08:38 UTC

[GitHub] [nifi-minifi-cpp] fgerlits commented on a diff in pull request #1424: MINIFICPP-1862 use std::filesystem::path instead of std::string where appropriate

fgerlits commented on code in PR #1424:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1424#discussion_r1035973706


##########
Windows.md:
##########
@@ -93,7 +93,7 @@ A basic working CMake configuration can be inferred from the `win_build_vs.bat`.
 ```
 mkdir build
 cd build
-cmake -G "Visual Studio 16 2019" -DINSTALLER_MERGE_MODULES=OFF -DENABLE_SQL=OFF -DCMAKE_BUILD_TYPE_INIT=Release -DCMAKE_BUILD_TYPE=Release -DWIN32=WIN32 -DENABLE_LIBRDKAFKA=OFF -DENABLE_JNI=OFF -DOPENSSL_OFF=OFF -DENABLE_COAP=OFF -DUSE_SHARED_LIBS=OFF -DDISABLE_CONTROLLER=ON  -DBUILD_ROCKSDB=ON -DFORCE_WINDOWS=ON -DUSE_SYSTEM_UUID=OFF -DDISABLE_LIBARCHIVE=OFF -DDISABLE_SCRIPTING=ON -DEXCLUDE_BOOST=ON -DENABLE_WEL=TRUE -DFAIL_ON_WARNINGS=OFF -DSKIP_TESTS=OFF ..

Review Comment:
   `EXCLUDE_BOOST` should be removed from `win_build_vs.bat`, too



##########
extensions/expression-language/tests/ExpressionLanguageTests.cpp:
##########
@@ -218,7 +216,7 @@ TEST_CASE("GetFile PutFile dynamic attribute", "[expressionLanguageTestGetFilePu
   plan->setProperty(extract_text, minifi::processors::ExtractText::Attribute.getName(), "extracted_attr_name");
   plan->addProcessor("LogAttribute", "LogAttribute", core::Relationship("success", "description"), true);
   auto put_file = plan->addProcessor("PutFile", "PutFile", core::Relationship("success", "description"), true);
-  plan->setProperty(put_file, minifi::processors::PutFile::Directory.getName(), out_dir + "/${extracted_attr_name}");
+  plan->setProperty(put_file, minifi::processors::PutFile::Directory.getName(), out_dir.string() + "/${extracted_attr_name}");

Review Comment:
   ```suggestion
     plan->setProperty(put_file, minifi::processors::PutFile::Directory.getName(), (out_dir / "${extracted_attr_name}").string());
   ```



##########
extensions/http-curl/tests/C2VerifyServeResults.cpp:
##########
@@ -82,8 +82,8 @@ class VerifyC2Server : public HTTPIntegrationBase {
   }
 
  protected:
-  std::string dir;
-  std::stringstream ss;
+  std::filesystem::path dir_;
+  std::filesystem::path path_;

Review Comment:
   I think `test_file_` would be a better name than `path_`



##########
extensions/http-curl/tests/HttpPostIntegrationTest.cpp:
##########
@@ -78,9 +78,9 @@ class HttpTestHarness : public HTTPIntegrationBase {
   }
 
  protected:
-  std::string dir;
-  std::stringstream ss;
-  TestController testController;
+  std::filesystem::path dir_;
+  std::filesystem::path path_;

Review Comment:
   here, too, I would call this `test_file_` (and everywhere where we currently have a `std::filesystem::path path_` field which contains the location of a test file)



##########
extensions/script/python/PythonCreator.h:
##########
@@ -140,26 +136,18 @@ class PythonCreator : public minifi::core::CoreComponent {
     return python_package;
   }
 
-  std::string getPath(const std::string &pythonscript) {
-    return std::filesystem::path(pythonscript).parent_path().string();
+  std::filesystem::path getFileName(const std::filesystem::path& python_script) {
+    return std::filesystem::path(python_script).filename();

Review Comment:
   this conversion is no longer needed:
   ```suggestion
       return python_script.filename();
   ```
   
   Even better, we could remove `getFileName()` and `getScriptName()` and call `filename()` and `stem()` instead.



##########
extensions/jni/jvm/JniReferenceObjects.h:
##########
@@ -244,23 +228,24 @@ class JniSession : public core::WeakReference {
   }
 
   bool prune() {
-    global_ff_objects_.erase(std::remove_if(global_ff_objects_.begin(), global_ff_objects_.end(), check_empty_ff()), global_ff_objects_.end());
+    const auto check_empty_ff = [](const std::shared_ptr<JniFlowFile>& flow_file) { return flow_file->empty(); };
+    global_ff_objects_.erase(std::remove_if(global_ff_objects_.begin(), global_ff_objects_.end(), check_empty_ff), global_ff_objects_.end());

Review Comment:
   this could be simplified with range-v3:
   ```suggestion
       ranges::remove_if(global_ff_objects_, [](const std::shared_ptr<JniFlowFile>& flow_file) { return flow_file->empty(); });
   ```
   
   (with an `#include "range/v3/algorithm/remove_if.hpp"`)



##########
extensions/script/tests/TestExecuteScriptProcessorWithLuaScript.cpp:
##########
@@ -421,8 +412,10 @@ TEST_CASE("Lua: Test Module Directory property", "[executescriptLuaModuleDirecto
                                           true);
 
   plan->setProperty(executeScript, minifi::processors::ExecuteScript::ScriptEngine.getName(), "lua");
-  plan->setProperty(executeScript, minifi::processors::ExecuteScript::ScriptFile.getName(), getScriptFullPath("foo_bar_processor.lua"));
-  plan->setProperty(executeScript, minifi::processors::ExecuteScript::ModuleDirectory.getName(), getScriptFullPath("foo_modules/foo.lua") + "," + getScriptFullPath("bar_modules"));
+  plan->setProperty(executeScript, minifi::processors::ExecuteScript::ScriptFile.getName(), script_files_directory / "foo_bar_processor.lua");
+  plan->setProperty(executeScript,
+                    minifi::processors::ExecuteScript::ModuleDirectory.getName(),
+                    (script_files_directory / "foo_modules/foo.lua").string() + "," + (script_files_directory / "bar_modules").string());

Review Comment:
   ```suggestion
                       (script_files_directory / "foo_modules" / "foo.lua").string() + "," + (script_files_directory / "bar_modules").string());
   ```



##########
extensions/sftp/processors/FetchSFTP.cpp:
##########
@@ -87,12 +87,11 @@ void FetchSFTP::onTrigger(const std::shared_ptr<core::ProcessContext> &context,
     return;
   }
 
-  /* Parse processor-specific properties */
-  std::string remote_file;
-  std::string move_destination_directory;
-
-  context->getProperty(RemoteFile, remote_file, flow_file);
-  context->getProperty(MoveDestinationDirectory, move_destination_directory, flow_file);
+  std::string path_str;
+  context->getProperty(RemoteFile, path_str, flow_file);
+  auto remote_file = std::filesystem::path(path_str, std::filesystem::path::format::generic_format);
+  context->getProperty(MoveDestinationDirectory, path_str, flow_file);

Review Comment:
   Reusing `path_str` is a bad idea, because if `RemoteFile` is set to `"foo"` but `MoveDestinationDirectory` is not set, then `move_destination_directory` will be set to `"foo"`, as well.



##########
extensions/script/tests/TestExecuteScriptProcessorWithPythonScript.cpp:
##########
@@ -295,124 +291,119 @@ TEST_CASE("Python: Test Update Attribute", "[executescriptPythonUpdateAttribute]
         session.transfer(flow_file, REL_SUCCESS)
   )");
 
-  auto getFileDir = testController.createTempDirectory();
-  plan->setProperty(getFile, minifi::processors::GetFile::Directory.getName(), getFileDir);
+  auto get_file_dir = test_controller.createTempDirectory();
+  plan->setProperty(getFile, minifi::processors::GetFile::Directory.getName(), get_file_dir.string());
 
-  utils::putFileToDir(getFileDir, "tstFile.ext", "tempFile");
+  utils::putFileToDir(get_file_dir, "tstFile.ext", "tempFile");
 
-  testController.runSession(plan, false);
-  testController.runSession(plan, false);
-  testController.runSession(plan, false);
+  test_controller.runSession(plan, false);
+  test_controller.runSession(plan, false);
+  test_controller.runSession(plan, false);
 
   REQUIRE(LogTestController::getInstance().contains("key:test_attr value:2"));
 
-  logTestController.reset();
+  log_test_controller.reset();
 }
 
 TEST_CASE("Python: Test Get Context Property", "[executescriptPythonGetContextProperty]") {
-  TestController testController;
+  TestController test_controller;
 
-  LogTestController &logTestController = LogTestController::getInstance();
-  logTestController.setDebug<TestPlan>();
-  logTestController.setDebug<minifi::processors::LogAttribute>();
-  logTestController.setDebug<minifi::processors::ExecuteScript>();
+  LogTestController& log_test_controller = LogTestController::getInstance();
+  log_test_controller.setDebug<TestPlan>();
+  log_test_controller.setDebug<minifi::processors::LogAttribute>();
+  log_test_controller.setDebug<minifi::processors::ExecuteScript>();
 
-  auto plan = testController.createPlan();
+  auto plan = test_controller.createPlan();
 
-  auto getFile = plan->addProcessor("GetFile", "getFile");
-  auto executeScript = plan->addProcessor("ExecuteScript",
+  auto get_file = plan->addProcessor("GetFile", "getFile");
+  auto execute_script = plan->addProcessor("ExecuteScript",
                                           "executeScript",
                                           core::Relationship("success", "description"),
                                           true);
-  auto logAttribute = plan->addProcessor("LogAttribute", "logAttribute",
+  auto log_attribute = plan->addProcessor("LogAttribute", "logAttribute",
                                          core::Relationship("success", "description"),
                                          true);
 
-  plan->setProperty(executeScript, minifi::processors::ExecuteScript::ScriptBody.getName(), R"(
+  plan->setProperty(execute_script, minifi::processors::ExecuteScript::ScriptBody.getName(), R"(
     def onTrigger(context, session):
       script_engine = context.getProperty('Script Engine')
       log.info('got Script Engine property: %s' % script_engine)
   )");
 
-  auto getFileDir = testController.createTempDirectory();
-  plan->setProperty(getFile, minifi::processors::GetFile::Directory.getName(), getFileDir);
+  auto get_file_dir = test_controller.createTempDirectory();
+  plan->setProperty(get_file, minifi::processors::GetFile::Directory.getName(), get_file_dir.string());
 
-  utils::putFileToDir(getFileDir, "tstFile.ext", "tempFile");
+  utils::putFileToDir(get_file_dir, "tstFile.ext", "tempFile");
 
-  testController.runSession(plan, false);
-  testController.runSession(plan, false);
-  testController.runSession(plan, false);
+  test_controller.runSession(plan, false);
+  test_controller.runSession(plan, false);
+  test_controller.runSession(plan, false);
 
   REQUIRE(LogTestController::getInstance().contains("[info] got Script Engine property: python"));
 
-  logTestController.reset();
+  log_test_controller.reset();
 }
 
 TEST_CASE("Python: Test Module Directory property", "[executescriptPythonModuleDirectoryProperty]") {
-  using org::apache::nifi::minifi::utils::file::concat_path;
   using org::apache::nifi::minifi::utils::file::get_executable_dir;
 
-  TestController testController;
+  TestController test_controller;
 
-  LogTestController &logTestController = LogTestController::getInstance();
-  logTestController.setDebug<TestPlan>();
-  logTestController.setDebug<minifi::processors::ExecuteScript>();
+  LogTestController& log_test_controller = LogTestController::getInstance();
+  log_test_controller.setDebug<TestPlan>();
+  log_test_controller.setDebug<minifi::processors::ExecuteScript>();
 
-  std::string path;
-  std::string filename;
-  minifi::utils::file::getFileNameAndPath(__FILE__, path, filename);
-  const std::string SCRIPT_FILES_DIRECTORY = minifi::utils::file::getFullPath(concat_path(path, "test_python_scripts"));
+  const auto script_files_directory = std::filesystem::path(__FILE__).parent_path() / "test_python_scripts";
 
-  auto getScriptFullPath = [&SCRIPT_FILES_DIRECTORY](const std::string& script_file_name) {
-    return concat_path(SCRIPT_FILES_DIRECTORY, script_file_name);
-  };
+  auto plan = test_controller.createPlan();
 
-  auto plan = testController.createPlan();
-
-  auto getFile = plan->addProcessor("GetFile", "getFile");
-  auto executeScript = plan->addProcessor("ExecuteScript",
+  auto get_file = plan->addProcessor("GetFile", "getFile");
+  auto execute_script = plan->addProcessor("ExecuteScript",
                                           "executeScript",
                                           core::Relationship("success", "description"),
                                           true);
 
-  plan->setProperty(executeScript, minifi::processors::ExecuteScript::ScriptFile.getName(), getScriptFullPath("foo_bar_processor.py"));
-  plan->setProperty(executeScript, minifi::processors::ExecuteScript::ModuleDirectory.getName(), getScriptFullPath(concat_path("foo_modules", "foo.py")) + "," + getScriptFullPath("bar_modules"));
+  plan->setProperty(execute_script, minifi::processors::ExecuteScript::ScriptEngine.getName(), "python");
+  plan->setProperty(execute_script, minifi::processors::ExecuteScript::ScriptFile.getName(), (script_files_directory / "foo_bar_processor.py").string());
+  plan->setProperty(execute_script,
+                    minifi::processors::ExecuteScript::ModuleDirectory.getName(),
+                    (script_files_directory / "foo_modules/foo.py").string() + "," + (script_files_directory / "bar_modules").string());

Review Comment:
   ```suggestion
                       (script_files_directory / "foo_modules" / "foo.py").string() + "," + (script_files_directory / "bar_modules").string());
   ```



##########
extensions/sftp/tests/ListSFTPTests.cpp:
##########
@@ -187,7 +180,7 @@ TEST_CASE_METHOD(ListSFTPTestsFixture, "ListSFTP list one file", "[ListSFTP][bas
 
 TEST_CASE_METHOD(ListSFTPTestsFixture, "ListSFTP public key authentication", "[ListSFTP][basic]") {
   plan->setProperty(list_sftp, "Remote File", "nifi_test/tstFile.ext");
-  plan->setProperty(list_sftp, "Private Key Path", utils::file::FileUtils::concat_path(get_sftp_test_dir(), "resources/id_rsa"));
+  plan->setProperty(list_sftp, "Private Key Path", get_sftp_test_dir() / "resources/id_rsa");

Review Comment:
   ```suggestion
     plan->setProperty(list_sftp, "Private Key Path", (get_sftp_test_dir() / "resources" / "id_rsa").string());
   ```



##########
extensions/sftp/tests/PutSFTPTests.cpp:
##########
@@ -244,7 +240,7 @@ TEST_CASE_METHOD(PutSFTPTestsFixture, "PutSFTP bad password", "[PutSFTP][authent
 }
 
 TEST_CASE_METHOD(PutSFTPTestsFixture, "PutSFTP public key authentication success", "[PutSFTP][authentication]") {
-  plan->setProperty(put, "Private Key Path", utils::file::concat_path(get_sftp_test_dir(), "resources/id_rsa"));
+  plan->setProperty(put, "Private Key Path", get_sftp_test_dir() / "resources/id_rsa");

Review Comment:
   there are many lines like this in this file where the `/` could be moved outside the `""`:
   ```suggestion
     plan->setProperty(put, "Private Key Path", get_sftp_test_dir() / "resources" / "id_rsa");
   ```



##########
extensions/standard-processors/processors/TailFile.cpp:
##########
@@ -530,21 +530,20 @@ bool TailFile::getStateFromStateManager(std::map<std::string, TailState> &new_ta
             readOptionalInt64(state_map, "file." + std::to_string(i) + ".last_read_time")
         }};
 
-        std::string fileLocation;
-        std::string fileName;
-        if (utils::file::getFileNameAndPath(current, fileLocation, fileName)) {
-          logger_->log_debug("Received path %s, file %s", fileLocation, fileName);
-          new_tail_states.emplace(current, TailState{fileLocation, fileName, position, last_read_time, checksum});
+        std::filesystem::path file_path = current;
+        if (file_path.has_filename() && file_path.has_parent_path()) {
+          logger_->log_debug("Received path %s, file %s", file_path.parent_path().string(), file_path.filename().string());
+          new_tail_states.emplace(current, TailState{file_path.parent_path(), file_path.filename(), position, last_read_time, checksum});
         } else {
-          new_tail_states.emplace(current, TailState{fileLocation, current, position, last_read_time, checksum});
+          new_tail_states.emplace(current, TailState{file_path.parent_path(), current, position, last_read_time, checksum});

Review Comment:
   very minor, but we have already converted `current` to a `path`, so no need to do it again:
   ```suggestion
             new_tail_states.emplace(current, TailState{file_path.parent_path(), file_path, position, last_read_time, checksum});
   ```



##########
extensions/standard-processors/tests/unit/ProcessorTests.cpp:
##########
@@ -197,10 +196,8 @@ TEST_CASE("Test GetFile Ignore", "[getfileCreate3]") {
   REQUIRE(record == nullptr);
   REQUIRE(records.empty());
 
-  const std::string hidden_file_name = [&] {
-    std::stringstream ss;
-    ss << dir << utils::file::FileUtils::get_separator() << ".filewithoutanext";
-    return ss.str();
+  const std::filesystem::path hidden_file_name = [&] {
+    return dir / ".filewithoutanext";
   }();

Review Comment:
   the lambda is not needed any longer:
   ```suggestion
     const std::filesystem::path hidden_file_name = dir / ".filewithoutanext";
   ```



##########
extensions/standard-processors/tests/unit/GenerateFlowFileTests.cpp:
##########
@@ -57,9 +57,9 @@ TEST_CASE("GenerateFlowFileTest", "[generateflowfiletest]") {
 
   std::vector<std::string> file_contents;
 
-  auto lambda = [&file_contents](const std::string& path, const std::string& filename) -> bool {
-    std::ifstream is(path + utils::file::get_separator() + filename, std::ifstream::binary);
-    file_contents.push_back(std::string((std::istreambuf_iterator<char>(is)), std::istreambuf_iterator<char>()));
+  auto lambda = [&file_contents](const std::filesystem::path& path, const std::filesystem::path& filename) -> bool {
+    std::ifstream is(path / filename, std::ifstream::binary);
+    file_contents.emplace_back((std::istreambuf_iterator<char>(is)), std::istreambuf_iterator<char>());

Review Comment:
   nitpicking, but I don't think the `(...)`s are needed:
   ```suggestion
       file_contents.emplace_back(std::istreambuf_iterator<char>(is), std::istreambuf_iterator<char>());
   ```
   (also in a few other places in this file)



##########
extensions/sftp/tests/ListSFTPTests.cpp:
##########
@@ -139,15 +139,8 @@ class ListSFTPTestsFixture {
     std::fstream file;
     std::stringstream ss;
     ss << src_dir << "/vfs/" << relative_path;
-    auto full_path = ss.str();
-    std::deque<std::string> parent_dirs;
-    std::string parent_dir = full_path;
-    while (!(parent_dir = utils::file::FileUtils::get_parent_path(parent_dir)).empty()) {
-      parent_dirs.push_front(parent_dir);
-    }
-    for (const auto& dir : parent_dirs) {
-      utils::file::FileUtils::create_dir(dir);
-    }
+    std::filesystem::path full_path = ss.str();

Review Comment:
   `src_dir` could be left as a `path`, and then this could be de-stringstreamed, too
   
   as it is, line 158 doesn't compile on Windows
   
   (same in `ListThenFetchSFTPTests`)



##########
extensions/standard-processors/processors/GetFile.cpp:
##########
@@ -154,36 +154,32 @@ void GetFile::onTrigger(core::ProcessContext* /*context*/, core::ProcessSession*
     return;
   }
 
-  std::queue<std::string> list_of_file_names = pollListing(request_.batchSize);
+  std::queue<std::filesystem::path> list_of_file_names = pollListing(request_.batchSize);
   while (!list_of_file_names.empty()) {
-    std::string file_name = list_of_file_names.front();
+    auto file_name = list_of_file_names.front();
     list_of_file_names.pop();
     getSingleFile(*session, file_name);
   }
 }
 
-void GetFile::getSingleFile(core::ProcessSession& session, const std::string& file_name) const {
-  logger_->log_info("GetFile process %s", file_name);
+void GetFile::getSingleFile(core::ProcessSession& session, const std::filesystem::path& file_path) const {
+  logger_->log_info("GetFile process %s", file_path.string());
   auto flow_file = session.create();
   gsl_Expects(flow_file);
-  std::string path;
-  std::string name;
-  std::tie(path, name) = utils::file::split_path(file_name);
-  flow_file->setAttribute(core::SpecialFlowAttribute::FILENAME, name);
-  flow_file->setAttribute(core::SpecialFlowAttribute::PATH, path);
-  flow_file->addAttribute(core::SpecialFlowAttribute::ABSOLUTE_PATH, file_name);
+  flow_file->setAttribute(core::SpecialFlowAttribute::FILENAME, file_path.filename().string());
+  flow_file->setAttribute(core::SpecialFlowAttribute::PATH, (file_path.parent_path() / "").string());
+  flow_file->addAttribute(core::SpecialFlowAttribute::ABSOLUTE_PATH, file_path.string());
 
   try {
-    session.write(flow_file, utils::FileReaderCallback{file_name});
+    session.write(flow_file, utils::FileReaderCallback{file_path});
     session.transfer(flow_file, Success);
     if (!request_.keepSourceFile) {
-      auto remove_status = remove(file_name.c_str());
-      if (remove_status != 0) {
-        logger_->log_error("GetFile could not delete file '%s', error %d: %s", file_name, errno, strerror(errno));
+      if (!std::filesystem::remove(file_path)) {
+        logger_->log_error("GetFile could not delete file '%s', error %d: %s", file_path.string(), errno, strerror(errno));

Review Comment:
   Are you sure `filesystem::remove()` sets `errno`?  I think we need to use the overload with `std::error_code`.
   
   (edit: as you did in `PutFile.cpp`)



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