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 2021/02/01 14:23:26 UTC

[GitHub] [nifi-minifi-cpp] martinzink opened a new pull request #987: MINIFICPP-1455: Fix FileStream error handling and reporting

martinzink opened a new pull request #987:
URL: https://github.com/apache/nifi-minifi-cpp/pull/987


   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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] martinzink commented on a change in pull request #987: MINIFICPP-1455: Fix FileStream error handling and reporting

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



##########
File path: libminifi/src/io/FileStream.cpp
##########
@@ -31,21 +31,41 @@ namespace nifi {
 namespace minifi {
 namespace io {
 
+constexpr const char *file_opening_error_msg = "Error opening file: ";
+constexpr const char *read_error_msg = "Error reading from file: ";
+constexpr const char *write_error_msg = "Error writing to file: ";
+constexpr const char *seek_error = "Error seeking in file: ";
+constexpr const char *invalid_file_stream_error_msg = "invalid file stream";
+constexpr const char *tellg_call_error_msg = "tellg call on file stream failed";
+constexpr const char *invalid_buffer_error_msg = "invalid buffer";
+constexpr const char *flush_call_error_msg = "flush call on file stream failed";
+constexpr const char *write_call_error_msg = "write call on file stream failed";
+constexpr const char *empty_message_error_msg = "empty message";
+constexpr const char *seekg_call_error_msg = "seekg call on file stream failed";
+constexpr const char *seekp_call_error_msg = "seekp call on file stream failed";

Review comment:
       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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] lordgamez commented on a change in pull request #987: MINIFICPP-1455: Fix FileStream error handling and reporting

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



##########
File path: libminifi/src/io/FileStream.cpp
##########
@@ -114,17 +151,18 @@ int FileStream::read(uint8_t *buf, int buflen) {
   }
   if (!IsNullOrEmpty(buf)) {
     std::lock_guard<std::mutex> lock(file_lock_);
-    if (!file_stream_) {
+    if (file_stream_ == nullptr || !file_stream_->is_open()) {
+      logging::LOG_ERROR(logger_) << read_error_msg << invalid_file_stream_error_msg;
       return -1;
     }
     file_stream_->read(reinterpret_cast<char*>(buf), buflen);
-    if ((file_stream_->rdstate() & (file_stream_->eofbit | file_stream_->failbit)) != 0) {
+    if (file_stream_->eof() || file_stream_->fail()) {
       file_stream_->clear();
       file_stream_->seekg(0, file_stream_->end);
       file_stream_->seekp(0, file_stream_->end);

Review comment:
       Maybe these could be checked as well.

##########
File path: libminifi/src/io/FileStream.cpp
##########
@@ -59,15 +79,19 @@ FileStream::FileStream(const std::string &path, uint32_t offset, bool write_enab
   } else {
     file_stream_->open(path.c_str(), std::fstream::in | std::fstream::binary);
   }
-  file_stream_->seekg(0, file_stream_->end);
-  file_stream_->seekp(0, file_stream_->end);
-  std::streamoff len = file_stream_->tellg();
-  if (len > 0) {
-    length_ = gsl::narrow<size_t>(len);
+  if (file_stream_->is_open()) {
+    file_stream_->seekg(0, file_stream_->end);
+    file_stream_->seekp(0, file_stream_->end);
+    std::streamoff len = file_stream_->tellg();
+    if (len > 0) {
+      length_ = gsl::narrow<size_t>(len);
+    } else {
+      length_ = 0;
+    }

Review comment:
       We could use the same ternary operator as above here as well.

##########
File path: libminifi/src/io/FileStream.cpp
##########
@@ -31,21 +31,41 @@ namespace nifi {
 namespace minifi {
 namespace io {
 
+constexpr const char *file_opening_error_msg = "Error opening file: ";
+constexpr const char *read_error_msg = "Error reading from file: ";
+constexpr const char *write_error_msg = "Error writing to file: ";
+constexpr const char *seek_error = "Error seeking in file: ";
+constexpr const char *invalid_file_stream_error_msg = "invalid file stream";
+constexpr const char *tellg_call_error_msg = "tellg call on file stream failed";
+constexpr const char *invalid_buffer_error_msg = "invalid buffer";
+constexpr const char *flush_call_error_msg = "flush call on file stream failed";
+constexpr const char *write_call_error_msg = "write call on file stream failed";
+constexpr const char *empty_message_error_msg = "empty message";
+constexpr const char *seekg_call_error_msg = "seekg call on file stream failed";
+constexpr const char *seekp_call_error_msg = "seekp call on file stream failed";
+
 FileStream::FileStream(const std::string &path, bool append)
     : logger_(logging::LoggerFactory<FileStream>::getLogger()),
       path_(path),
       offset_(0) {
   file_stream_ = std::unique_ptr<std::fstream>(new std::fstream());
   if (append) {
     file_stream_->open(path.c_str(), std::fstream::in | std::fstream::out | std::fstream::app | std::fstream::binary);
-    file_stream_->seekg(0, file_stream_->end);
-    file_stream_->seekp(0, file_stream_->end);
-    std::streamoff len = file_stream_->tellg();
-    length_ = len > 0 ? gsl::narrow<size_t>(len) : 0;
-    seek(offset_);
+    if (file_stream_->is_open()) {
+      file_stream_->seekg(0, file_stream_->end);
+      file_stream_->seekp(0, file_stream_->end);
+      std::streamoff len = file_stream_->tellg();

Review comment:
       Should we maybe check if seekg, seekp and tellg were successful as well?

##########
File path: libminifi/test/unit/FileStreamTests.cpp
##########
@@ -263,3 +267,75 @@ TEST_CASE("Read zero bytes") {
   minifi::io::FileStream stream(utils::file::concat_path(dir, "test.txt"), 0, true);
   REQUIRE(stream.read(nullptr, 0) == 0);
 }
+
+TEST_CASE("Non-existing file read/write test") {
+  TestController test_controller;
+  char format[] = "/tmp/gt.XXXXXX";
+  auto dir = test_controller.createTempDirectory(format);
+  minifi::io::FileStream stream(utils::file::concat_path(dir, "non_existing_file.txt"), 0, true);
+  REQUIRE(test_controller.getLog().getInstance().contains("Error opening file", std::chrono::seconds(0)));
+  REQUIRE(test_controller.getLog().getInstance().contains("No such file or directory", std::chrono::seconds(0)));
+  REQUIRE(stream.write("lorem ipsum", false) == -1);
+  REQUIRE(test_controller.getLog().getInstance().contains("Error writing to file: invalid file stream", std::chrono::seconds(0)));
+  std::vector<uint8_t> readBuffer;
+  stream.seek(0);
+  REQUIRE(stream.read(readBuffer, 1) == -1);
+  REQUIRE(test_controller.getLog().getInstance().contains("Error reading from file: invalid file stream", std::chrono::seconds(0)));
+}
+
+TEST_CASE("Existing file read/write test") {
+  TestController test_controller;
+  char format[] = "/tmp/gt.XXXXXX";
+  auto dir = test_controller.createTempDirectory(format);
+  std::string path_to_existing_file(utils::file::concat_path(dir, "existing_file.txt"));
+  {
+    std::ofstream outfile(path_to_existing_file);
+    outfile << "lorem ipsum" << std::endl;
+    outfile.close();
+  }
+  minifi::io::FileStream stream(path_to_existing_file, 0, true);
+  REQUIRE_FALSE(test_controller.getLog().getInstance().contains("Error opening file", std::chrono::seconds(0)));
+  REQUIRE_FALSE(stream.write("dolor sit amet", false) == -1);
+  REQUIRE_FALSE(test_controller.getLog().getInstance().contains("Error writing to file", std::chrono::seconds(0)));
+  std::vector<uint8_t> readBuffer;
+  stream.seek(0);
+  REQUIRE_FALSE(stream.read(readBuffer, 11) == -1);
+  REQUIRE_FALSE(test_controller.getLog().getInstance().contains("Error reading from file", std::chrono::seconds(0)));
+  stream.seek(0);
+  REQUIRE(stream.read(nullptr, 11) == -1);
+  REQUIRE(test_controller.getLog().getInstance().contains("Error reading from file: invalid buffer", std::chrono::seconds(0)));
+}
+
+#ifdef USE_BOOST
+TEST_CASE("Opening file without permission creates error logs") {
+  TestController test_controller;
+  char format[] = "/tmp/gt.XXXXXX";
+  auto dir = test_controller.createTempDirectory(format);
+  std::string path_to_permissionless_file(utils::file::concat_path(dir, "permissionless_file.txt"));
+  {
+    std::ofstream outfile(path_to_permissionless_file);
+    outfile << "this file has been just created" << std::endl;
+    outfile.close();
+    // This could be done with C++17 std::filesystem
+    boost::filesystem::permissions(path_to_permissionless_file, boost::filesystem::no_perms);

Review comment:
       We could use `set_permissions` functions from `FileUtils.h`, but then the test needs to be undefined for WIN32.

##########
File path: libminifi/src/io/FileStream.cpp
##########
@@ -31,21 +31,41 @@ namespace nifi {
 namespace minifi {
 namespace io {
 
+constexpr const char *file_opening_error_msg = "Error opening file: ";
+constexpr const char *read_error_msg = "Error reading from file: ";
+constexpr const char *write_error_msg = "Error writing to file: ";
+constexpr const char *seek_error = "Error seeking in file: ";
+constexpr const char *invalid_file_stream_error_msg = "invalid file stream";
+constexpr const char *tellg_call_error_msg = "tellg call on file stream failed";
+constexpr const char *invalid_buffer_error_msg = "invalid buffer";
+constexpr const char *flush_call_error_msg = "flush call on file stream failed";
+constexpr const char *write_call_error_msg = "write call on file stream failed";
+constexpr const char *empty_message_error_msg = "empty message";
+constexpr const char *seekg_call_error_msg = "seekg call on file stream failed";
+constexpr const char *seekp_call_error_msg = "seekp call on file stream failed";

Review comment:
       I could not find it in the guideline, but we usually have a FULL_CAPS format for constant variables.

##########
File path: libminifi/src/io/FileStream.cpp
##########
@@ -77,10 +101,16 @@ void FileStream::close() {
 
 void FileStream::seek(uint64_t offset) {
   std::lock_guard<std::mutex> lock(file_lock_);
+  if (file_stream_ == nullptr || !file_stream_->is_open()) {
+    logging::LOG_ERROR(logger_) << seek_error << invalid_file_stream_error_msg;
+    return;
+  }
   offset_ = gsl::narrow<size_t>(offset);
   file_stream_->clear();
-  file_stream_->seekg(offset_);
-  file_stream_->seekp(offset_);
+  if (!file_stream_->seekg(offset_))
+    logging::LOG_ERROR(logger_) << seek_error << seekg_call_error_msg;

Review comment:
       Should we return here if it fails?




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] martinzink commented on a change in pull request #987: MINIFICPP-1455: Fix FileStream error handling and reporting

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



##########
File path: libminifi/src/io/FileStream.cpp
##########
@@ -31,21 +31,41 @@ namespace nifi {
 namespace minifi {
 namespace io {
 
+constexpr const char *file_opening_error_msg = "Error opening file: ";
+constexpr const char *read_error_msg = "Error reading from file: ";
+constexpr const char *write_error_msg = "Error writing to file: ";
+constexpr const char *seek_error = "Error seeking in file: ";
+constexpr const char *invalid_file_stream_error_msg = "invalid file stream";
+constexpr const char *tellg_call_error_msg = "tellg call on file stream failed";
+constexpr const char *invalid_buffer_error_msg = "invalid buffer";
+constexpr const char *flush_call_error_msg = "flush call on file stream failed";
+constexpr const char *write_call_error_msg = "write call on file stream failed";
+constexpr const char *empty_message_error_msg = "empty message";
+constexpr const char *seekg_call_error_msg = "seekg call on file stream failed";
+constexpr const char *seekp_call_error_msg = "seekp call on file stream failed";

Review comment:
       fixed

##########
File path: libminifi/src/io/FileStream.cpp
##########
@@ -31,21 +31,41 @@ namespace nifi {
 namespace minifi {
 namespace io {
 
+constexpr const char *file_opening_error_msg = "Error opening file: ";
+constexpr const char *read_error_msg = "Error reading from file: ";
+constexpr const char *write_error_msg = "Error writing to file: ";
+constexpr const char *seek_error = "Error seeking in file: ";
+constexpr const char *invalid_file_stream_error_msg = "invalid file stream";
+constexpr const char *tellg_call_error_msg = "tellg call on file stream failed";
+constexpr const char *invalid_buffer_error_msg = "invalid buffer";
+constexpr const char *flush_call_error_msg = "flush call on file stream failed";
+constexpr const char *write_call_error_msg = "write call on file stream failed";
+constexpr const char *empty_message_error_msg = "empty message";
+constexpr const char *seekg_call_error_msg = "seekg call on file stream failed";
+constexpr const char *seekp_call_error_msg = "seekp call on file stream failed";
+
 FileStream::FileStream(const std::string &path, bool append)
     : logger_(logging::LoggerFactory<FileStream>::getLogger()),
       path_(path),
       offset_(0) {
   file_stream_ = std::unique_ptr<std::fstream>(new std::fstream());
   if (append) {
     file_stream_->open(path.c_str(), std::fstream::in | std::fstream::out | std::fstream::app | std::fstream::binary);
-    file_stream_->seekg(0, file_stream_->end);
-    file_stream_->seekp(0, file_stream_->end);
-    std::streamoff len = file_stream_->tellg();
-    length_ = len > 0 ? gsl::narrow<size_t>(len) : 0;
-    seek(offset_);
+    if (file_stream_->is_open()) {
+      file_stream_->seekg(0, file_stream_->end);
+      file_stream_->seekp(0, file_stream_->end);
+      std::streamoff len = file_stream_->tellg();

Review comment:
       agreed, and since seeking to the end of the file is used multiple times, I moved them to a private function (with error logging)

##########
File path: libminifi/src/io/FileStream.cpp
##########
@@ -59,15 +79,19 @@ FileStream::FileStream(const std::string &path, uint32_t offset, bool write_enab
   } else {
     file_stream_->open(path.c_str(), std::fstream::in | std::fstream::binary);
   }
-  file_stream_->seekg(0, file_stream_->end);
-  file_stream_->seekp(0, file_stream_->end);
-  std::streamoff len = file_stream_->tellg();
-  if (len > 0) {
-    length_ = gsl::narrow<size_t>(len);
+  if (file_stream_->is_open()) {
+    file_stream_->seekg(0, file_stream_->end);
+    file_stream_->seekp(0, file_stream_->end);
+    std::streamoff len = file_stream_->tellg();
+    if (len > 0) {
+      length_ = gsl::narrow<size_t>(len);
+    } else {
+      length_ = 0;
+    }

Review comment:
       done

##########
File path: libminifi/src/io/FileStream.cpp
##########
@@ -77,10 +101,16 @@ void FileStream::close() {
 
 void FileStream::seek(uint64_t offset) {
   std::lock_guard<std::mutex> lock(file_lock_);
+  if (file_stream_ == nullptr || !file_stream_->is_open()) {
+    logging::LOG_ERROR(logger_) << seek_error << invalid_file_stream_error_msg;
+    return;
+  }
   offset_ = gsl::narrow<size_t>(offset);
   file_stream_->clear();
-  file_stream_->seekg(offset_);
-  file_stream_->seekp(offset_);
+  if (!file_stream_->seekg(offset_))
+    logging::LOG_ERROR(logger_) << seek_error << seekg_call_error_msg;

Review comment:
       That is a good question. I felt the same at first, but since the read and write pointers should be pointing to the same position, I think it would be better to call them both regardless of errors we encounter in the first call.

##########
File path: libminifi/src/io/FileStream.cpp
##########
@@ -114,17 +151,18 @@ int FileStream::read(uint8_t *buf, int buflen) {
   }
   if (!IsNullOrEmpty(buf)) {
     std::lock_guard<std::mutex> lock(file_lock_);
-    if (!file_stream_) {
+    if (file_stream_ == nullptr || !file_stream_->is_open()) {
+      logging::LOG_ERROR(logger_) << read_error_msg << invalid_file_stream_error_msg;
       return -1;
     }
     file_stream_->read(reinterpret_cast<char*>(buf), buflen);
-    if ((file_stream_->rdstate() & (file_stream_->eofbit | file_stream_->failbit)) != 0) {
+    if (file_stream_->eof() || file_stream_->fail()) {
       file_stream_->clear();
       file_stream_->seekg(0, file_stream_->end);
       file_stream_->seekp(0, file_stream_->end);

Review comment:
       done

##########
File path: libminifi/test/unit/FileStreamTests.cpp
##########
@@ -263,3 +267,75 @@ TEST_CASE("Read zero bytes") {
   minifi::io::FileStream stream(utils::file::concat_path(dir, "test.txt"), 0, true);
   REQUIRE(stream.read(nullptr, 0) == 0);
 }
+
+TEST_CASE("Non-existing file read/write test") {
+  TestController test_controller;
+  char format[] = "/tmp/gt.XXXXXX";
+  auto dir = test_controller.createTempDirectory(format);
+  minifi::io::FileStream stream(utils::file::concat_path(dir, "non_existing_file.txt"), 0, true);
+  REQUIRE(test_controller.getLog().getInstance().contains("Error opening file", std::chrono::seconds(0)));
+  REQUIRE(test_controller.getLog().getInstance().contains("No such file or directory", std::chrono::seconds(0)));
+  REQUIRE(stream.write("lorem ipsum", false) == -1);
+  REQUIRE(test_controller.getLog().getInstance().contains("Error writing to file: invalid file stream", std::chrono::seconds(0)));
+  std::vector<uint8_t> readBuffer;
+  stream.seek(0);
+  REQUIRE(stream.read(readBuffer, 1) == -1);
+  REQUIRE(test_controller.getLog().getInstance().contains("Error reading from file: invalid file stream", std::chrono::seconds(0)));
+}
+
+TEST_CASE("Existing file read/write test") {
+  TestController test_controller;
+  char format[] = "/tmp/gt.XXXXXX";
+  auto dir = test_controller.createTempDirectory(format);
+  std::string path_to_existing_file(utils::file::concat_path(dir, "existing_file.txt"));
+  {
+    std::ofstream outfile(path_to_existing_file);
+    outfile << "lorem ipsum" << std::endl;
+    outfile.close();
+  }
+  minifi::io::FileStream stream(path_to_existing_file, 0, true);
+  REQUIRE_FALSE(test_controller.getLog().getInstance().contains("Error opening file", std::chrono::seconds(0)));
+  REQUIRE_FALSE(stream.write("dolor sit amet", false) == -1);
+  REQUIRE_FALSE(test_controller.getLog().getInstance().contains("Error writing to file", std::chrono::seconds(0)));
+  std::vector<uint8_t> readBuffer;
+  stream.seek(0);
+  REQUIRE_FALSE(stream.read(readBuffer, 11) == -1);
+  REQUIRE_FALSE(test_controller.getLog().getInstance().contains("Error reading from file", std::chrono::seconds(0)));
+  stream.seek(0);
+  REQUIRE(stream.read(nullptr, 11) == -1);
+  REQUIRE(test_controller.getLog().getInstance().contains("Error reading from file: invalid buffer", std::chrono::seconds(0)));
+}
+
+#ifdef USE_BOOST
+TEST_CASE("Opening file without permission creates error logs") {
+  TestController test_controller;
+  char format[] = "/tmp/gt.XXXXXX";
+  auto dir = test_controller.createTempDirectory(format);
+  std::string path_to_permissionless_file(utils::file::concat_path(dir, "permissionless_file.txt"));
+  {
+    std::ofstream outfile(path_to_permissionless_file);
+    outfile << "this file has been just created" << std::endl;
+    outfile.close();
+    // This could be done with C++17 std::filesystem
+    boost::filesystem::permissions(path_to_permissionless_file, boost::filesystem::no_perms);

Review comment:
       I feel like its useful to run this on windows as well (if there is boost support), because it verifies that the error messages provided by strerror() are the same on all platforms. (I wasn't sure about it previously given how differently windows and unix handles file permissions).
   Maybe we should do both, if there is boost use this if not and we are not on windows use the FileUtils one (which would then use chmod)?

##########
File path: libminifi/test/unit/FileStreamTests.cpp
##########
@@ -263,3 +267,75 @@ TEST_CASE("Read zero bytes") {
   minifi::io::FileStream stream(utils::file::concat_path(dir, "test.txt"), 0, true);
   REQUIRE(stream.read(nullptr, 0) == 0);
 }
+
+TEST_CASE("Non-existing file read/write test") {
+  TestController test_controller;
+  char format[] = "/tmp/gt.XXXXXX";
+  auto dir = test_controller.createTempDirectory(format);
+  minifi::io::FileStream stream(utils::file::concat_path(dir, "non_existing_file.txt"), 0, true);
+  REQUIRE(test_controller.getLog().getInstance().contains("Error opening file", std::chrono::seconds(0)));
+  REQUIRE(test_controller.getLog().getInstance().contains("No such file or directory", std::chrono::seconds(0)));
+  REQUIRE(stream.write("lorem ipsum", false) == -1);
+  REQUIRE(test_controller.getLog().getInstance().contains("Error writing to file: invalid file stream", std::chrono::seconds(0)));
+  std::vector<uint8_t> readBuffer;
+  stream.seek(0);
+  REQUIRE(stream.read(readBuffer, 1) == -1);
+  REQUIRE(test_controller.getLog().getInstance().contains("Error reading from file: invalid file stream", std::chrono::seconds(0)));
+}
+
+TEST_CASE("Existing file read/write test") {
+  TestController test_controller;
+  char format[] = "/tmp/gt.XXXXXX";
+  auto dir = test_controller.createTempDirectory(format);
+  std::string path_to_existing_file(utils::file::concat_path(dir, "existing_file.txt"));
+  {
+    std::ofstream outfile(path_to_existing_file);
+    outfile << "lorem ipsum" << std::endl;
+    outfile.close();
+  }
+  minifi::io::FileStream stream(path_to_existing_file, 0, true);
+  REQUIRE_FALSE(test_controller.getLog().getInstance().contains("Error opening file", std::chrono::seconds(0)));
+  REQUIRE_FALSE(stream.write("dolor sit amet", false) == -1);
+  REQUIRE_FALSE(test_controller.getLog().getInstance().contains("Error writing to file", std::chrono::seconds(0)));
+  std::vector<uint8_t> readBuffer;
+  stream.seek(0);
+  REQUIRE_FALSE(stream.read(readBuffer, 11) == -1);
+  REQUIRE_FALSE(test_controller.getLog().getInstance().contains("Error reading from file", std::chrono::seconds(0)));
+  stream.seek(0);
+  REQUIRE(stream.read(nullptr, 11) == -1);
+  REQUIRE(test_controller.getLog().getInstance().contains("Error reading from file: invalid buffer", std::chrono::seconds(0)));
+}
+
+#ifdef USE_BOOST
+TEST_CASE("Opening file without permission creates error logs") {
+  TestController test_controller;
+  char format[] = "/tmp/gt.XXXXXX";
+  auto dir = test_controller.createTempDirectory(format);
+  std::string path_to_permissionless_file(utils::file::concat_path(dir, "permissionless_file.txt"));
+  {
+    std::ofstream outfile(path_to_permissionless_file);
+    outfile << "this file has been just created" << std::endl;
+    outfile.close();
+    // This could be done with C++17 std::filesystem
+    boost::filesystem::permissions(path_to_permissionless_file, boost::filesystem::no_perms);

Review comment:
       changed it in c4f9a4e




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] lordgamez commented on a change in pull request #987: MINIFICPP-1455: Fix FileStream error handling and reporting

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



##########
File path: libminifi/test/unit/FileStreamTests.cpp
##########
@@ -263,3 +267,75 @@ TEST_CASE("Read zero bytes") {
   minifi::io::FileStream stream(utils::file::concat_path(dir, "test.txt"), 0, true);
   REQUIRE(stream.read(nullptr, 0) == 0);
 }
+
+TEST_CASE("Non-existing file read/write test") {
+  TestController test_controller;
+  char format[] = "/tmp/gt.XXXXXX";
+  auto dir = test_controller.createTempDirectory(format);
+  minifi::io::FileStream stream(utils::file::concat_path(dir, "non_existing_file.txt"), 0, true);
+  REQUIRE(test_controller.getLog().getInstance().contains("Error opening file", std::chrono::seconds(0)));
+  REQUIRE(test_controller.getLog().getInstance().contains("No such file or directory", std::chrono::seconds(0)));
+  REQUIRE(stream.write("lorem ipsum", false) == -1);
+  REQUIRE(test_controller.getLog().getInstance().contains("Error writing to file: invalid file stream", std::chrono::seconds(0)));
+  std::vector<uint8_t> readBuffer;
+  stream.seek(0);
+  REQUIRE(stream.read(readBuffer, 1) == -1);
+  REQUIRE(test_controller.getLog().getInstance().contains("Error reading from file: invalid file stream", std::chrono::seconds(0)));
+}
+
+TEST_CASE("Existing file read/write test") {
+  TestController test_controller;
+  char format[] = "/tmp/gt.XXXXXX";
+  auto dir = test_controller.createTempDirectory(format);
+  std::string path_to_existing_file(utils::file::concat_path(dir, "existing_file.txt"));
+  {
+    std::ofstream outfile(path_to_existing_file);
+    outfile << "lorem ipsum" << std::endl;
+    outfile.close();
+  }
+  minifi::io::FileStream stream(path_to_existing_file, 0, true);
+  REQUIRE_FALSE(test_controller.getLog().getInstance().contains("Error opening file", std::chrono::seconds(0)));
+  REQUIRE_FALSE(stream.write("dolor sit amet", false) == -1);
+  REQUIRE_FALSE(test_controller.getLog().getInstance().contains("Error writing to file", std::chrono::seconds(0)));
+  std::vector<uint8_t> readBuffer;
+  stream.seek(0);
+  REQUIRE_FALSE(stream.read(readBuffer, 11) == -1);
+  REQUIRE_FALSE(test_controller.getLog().getInstance().contains("Error reading from file", std::chrono::seconds(0)));
+  stream.seek(0);
+  REQUIRE(stream.read(nullptr, 11) == -1);
+  REQUIRE(test_controller.getLog().getInstance().contains("Error reading from file: invalid buffer", std::chrono::seconds(0)));
+}
+
+#ifdef USE_BOOST
+TEST_CASE("Opening file without permission creates error logs") {
+  TestController test_controller;
+  char format[] = "/tmp/gt.XXXXXX";
+  auto dir = test_controller.createTempDirectory(format);
+  std::string path_to_permissionless_file(utils::file::concat_path(dir, "permissionless_file.txt"));
+  {
+    std::ofstream outfile(path_to_permissionless_file);
+    outfile << "this file has been just created" << std::endl;
+    outfile.close();
+    // This could be done with C++17 std::filesystem
+    boost::filesystem::permissions(path_to_permissionless_file, boost::filesystem::no_perms);

Review comment:
       I think that's a good idea trying both. I just really didn't insist on running it on Windows as currently we do not have boost installed on our Windows CI runners.

##########
File path: libminifi/src/io/FileStream.cpp
##########
@@ -77,10 +101,16 @@ void FileStream::close() {
 
 void FileStream::seek(uint64_t offset) {
   std::lock_guard<std::mutex> lock(file_lock_);
+  if (file_stream_ == nullptr || !file_stream_->is_open()) {
+    logging::LOG_ERROR(logger_) << seek_error << invalid_file_stream_error_msg;
+    return;
+  }
   offset_ = gsl::narrow<size_t>(offset);
   file_stream_->clear();
-  file_stream_->seekg(offset_);
-  file_stream_->seekp(offset_);
+  if (!file_stream_->seekg(offset_))
+    logging::LOG_ERROR(logger_) << seek_error << seekg_call_error_msg;

Review comment:
       Okay, makes sense




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] martinzink commented on a change in pull request #987: MINIFICPP-1455: Fix FileStream error handling and reporting

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



##########
File path: libminifi/test/unit/FileStreamTests.cpp
##########
@@ -263,3 +267,75 @@ TEST_CASE("Read zero bytes") {
   minifi::io::FileStream stream(utils::file::concat_path(dir, "test.txt"), 0, true);
   REQUIRE(stream.read(nullptr, 0) == 0);
 }
+
+TEST_CASE("Non-existing file read/write test") {
+  TestController test_controller;
+  char format[] = "/tmp/gt.XXXXXX";
+  auto dir = test_controller.createTempDirectory(format);
+  minifi::io::FileStream stream(utils::file::concat_path(dir, "non_existing_file.txt"), 0, true);
+  REQUIRE(test_controller.getLog().getInstance().contains("Error opening file", std::chrono::seconds(0)));
+  REQUIRE(test_controller.getLog().getInstance().contains("No such file or directory", std::chrono::seconds(0)));
+  REQUIRE(stream.write("lorem ipsum", false) == -1);
+  REQUIRE(test_controller.getLog().getInstance().contains("Error writing to file: invalid file stream", std::chrono::seconds(0)));
+  std::vector<uint8_t> readBuffer;
+  stream.seek(0);
+  REQUIRE(stream.read(readBuffer, 1) == -1);
+  REQUIRE(test_controller.getLog().getInstance().contains("Error reading from file: invalid file stream", std::chrono::seconds(0)));
+}
+
+TEST_CASE("Existing file read/write test") {
+  TestController test_controller;
+  char format[] = "/tmp/gt.XXXXXX";
+  auto dir = test_controller.createTempDirectory(format);
+  std::string path_to_existing_file(utils::file::concat_path(dir, "existing_file.txt"));
+  {
+    std::ofstream outfile(path_to_existing_file);
+    outfile << "lorem ipsum" << std::endl;
+    outfile.close();
+  }
+  minifi::io::FileStream stream(path_to_existing_file, 0, true);
+  REQUIRE_FALSE(test_controller.getLog().getInstance().contains("Error opening file", std::chrono::seconds(0)));
+  REQUIRE_FALSE(stream.write("dolor sit amet", false) == -1);
+  REQUIRE_FALSE(test_controller.getLog().getInstance().contains("Error writing to file", std::chrono::seconds(0)));
+  std::vector<uint8_t> readBuffer;
+  stream.seek(0);
+  REQUIRE_FALSE(stream.read(readBuffer, 11) == -1);
+  REQUIRE_FALSE(test_controller.getLog().getInstance().contains("Error reading from file", std::chrono::seconds(0)));
+  stream.seek(0);
+  REQUIRE(stream.read(nullptr, 11) == -1);
+  REQUIRE(test_controller.getLog().getInstance().contains("Error reading from file: invalid buffer", std::chrono::seconds(0)));
+}
+
+#ifdef USE_BOOST
+TEST_CASE("Opening file without permission creates error logs") {
+  TestController test_controller;
+  char format[] = "/tmp/gt.XXXXXX";
+  auto dir = test_controller.createTempDirectory(format);
+  std::string path_to_permissionless_file(utils::file::concat_path(dir, "permissionless_file.txt"));
+  {
+    std::ofstream outfile(path_to_permissionless_file);
+    outfile << "this file has been just created" << std::endl;
+    outfile.close();
+    // This could be done with C++17 std::filesystem
+    boost::filesystem::permissions(path_to_permissionless_file, boost::filesystem::no_perms);

Review comment:
       changed it in c4f9a4e




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] arpadboda closed pull request #987: MINIFICPP-1455: Fix FileStream error handling and reporting

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


   


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] martinzink commented on a change in pull request #987: MINIFICPP-1455: Fix FileStream error handling and reporting

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



##########
File path: libminifi/src/io/FileStream.cpp
##########
@@ -31,21 +31,41 @@ namespace nifi {
 namespace minifi {
 namespace io {
 
+constexpr const char *file_opening_error_msg = "Error opening file: ";
+constexpr const char *read_error_msg = "Error reading from file: ";
+constexpr const char *write_error_msg = "Error writing to file: ";
+constexpr const char *seek_error = "Error seeking in file: ";
+constexpr const char *invalid_file_stream_error_msg = "invalid file stream";
+constexpr const char *tellg_call_error_msg = "tellg call on file stream failed";
+constexpr const char *invalid_buffer_error_msg = "invalid buffer";
+constexpr const char *flush_call_error_msg = "flush call on file stream failed";
+constexpr const char *write_call_error_msg = "write call on file stream failed";
+constexpr const char *empty_message_error_msg = "empty message";
+constexpr const char *seekg_call_error_msg = "seekg call on file stream failed";
+constexpr const char *seekp_call_error_msg = "seekp call on file stream failed";
+
 FileStream::FileStream(const std::string &path, bool append)
     : logger_(logging::LoggerFactory<FileStream>::getLogger()),
       path_(path),
       offset_(0) {
   file_stream_ = std::unique_ptr<std::fstream>(new std::fstream());
   if (append) {
     file_stream_->open(path.c_str(), std::fstream::in | std::fstream::out | std::fstream::app | std::fstream::binary);
-    file_stream_->seekg(0, file_stream_->end);
-    file_stream_->seekp(0, file_stream_->end);
-    std::streamoff len = file_stream_->tellg();
-    length_ = len > 0 ? gsl::narrow<size_t>(len) : 0;
-    seek(offset_);
+    if (file_stream_->is_open()) {
+      file_stream_->seekg(0, file_stream_->end);
+      file_stream_->seekp(0, file_stream_->end);
+      std::streamoff len = file_stream_->tellg();

Review comment:
       agreed, and since seeking to the end of the file is used multiple times, I moved them to a private function (with error logging)




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a change in pull request #987: MINIFICPP-1455: Fix FileStream error handling and reporting

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



##########
File path: libminifi/src/io/FileStream.cpp
##########
@@ -31,21 +31,42 @@ namespace nifi {
 namespace minifi {
 namespace io {
 
+constexpr const char *FILE_OPENING_ERROR_MSG = "Error opening file: ";
+constexpr const char *READ_ERROR_MSG = "Error reading from file: ";
+constexpr const char *WRITE_ERROR_MSG = "Error writing to file: ";
+constexpr const char *SEEK_ERROR_MSG = "Error seeking in file: ";
+constexpr const char *INVALID_FILE_STREAM_ERROR_MSG = "invalid file stream";
+constexpr const char *TELLG_CALL_ERROR_MSG = "tellg call on file stream failed";
+constexpr const char *INVALID_BUFFER_ERROR_MSG = "invalid buffer";
+constexpr const char *FLUSH_CALL_ERROR_MSG = "flush call on file stream failed";
+constexpr const char *WRITE_CALL_ERROR_MSG = "write call on file stream failed";
+constexpr const char *EMPTY_MESSAGE_ERROR_MSG = "empty message";
+constexpr const char *SEEKG_CALL_ERROR_MSG = "seekg call on file stream failed";
+constexpr const char *SEEKP_CALL_ERROR_MSG = "seekp call on file stream failed";
+
 FileStream::FileStream(const std::string &path, bool append)
     : logger_(logging::LoggerFactory<FileStream>::getLogger()),
       path_(path),
       offset_(0) {
   file_stream_ = std::unique_ptr<std::fstream>(new std::fstream());
   if (append) {
     file_stream_->open(path.c_str(), std::fstream::in | std::fstream::out | std::fstream::app | std::fstream::binary);
-    file_stream_->seekg(0, file_stream_->end);
-    file_stream_->seekp(0, file_stream_->end);
-    std::streamoff len = file_stream_->tellg();
-    length_ = len > 0 ? gsl::narrow<size_t>(len) : 0;
-    seek(offset_);
+    if (file_stream_->is_open()) {
+      seekToEndOfFile(FILE_OPENING_ERROR_MSG);
+      auto len = file_stream_->tellg();
+      if (len < 0)

Review comment:
       as I understand the actual return type of `tellg` is implementation defined and might be unsigned, we should compare the result to `pos_type(-1)` to detect errors




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] martinzink commented on a change in pull request #987: MINIFICPP-1455: Fix FileStream error handling and reporting

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



##########
File path: libminifi/src/io/FileStream.cpp
##########
@@ -59,15 +79,19 @@ FileStream::FileStream(const std::string &path, uint32_t offset, bool write_enab
   } else {
     file_stream_->open(path.c_str(), std::fstream::in | std::fstream::binary);
   }
-  file_stream_->seekg(0, file_stream_->end);
-  file_stream_->seekp(0, file_stream_->end);
-  std::streamoff len = file_stream_->tellg();
-  if (len > 0) {
-    length_ = gsl::narrow<size_t>(len);
+  if (file_stream_->is_open()) {
+    file_stream_->seekg(0, file_stream_->end);
+    file_stream_->seekp(0, file_stream_->end);
+    std::streamoff len = file_stream_->tellg();
+    if (len > 0) {
+      length_ = gsl::narrow<size_t>(len);
+    } else {
+      length_ = 0;
+    }

Review comment:
       done




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] lordgamez commented on a change in pull request #987: MINIFICPP-1455: Fix FileStream error handling and reporting

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



##########
File path: libminifi/src/io/FileStream.cpp
##########
@@ -77,10 +101,16 @@ void FileStream::close() {
 
 void FileStream::seek(uint64_t offset) {
   std::lock_guard<std::mutex> lock(file_lock_);
+  if (file_stream_ == nullptr || !file_stream_->is_open()) {
+    logging::LOG_ERROR(logger_) << seek_error << invalid_file_stream_error_msg;
+    return;
+  }
   offset_ = gsl::narrow<size_t>(offset);
   file_stream_->clear();
-  file_stream_->seekg(offset_);
-  file_stream_->seekp(offset_);
+  if (!file_stream_->seekg(offset_))
+    logging::LOG_ERROR(logger_) << seek_error << seekg_call_error_msg;

Review comment:
       Okay, makes sense




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] martinzink commented on a change in pull request #987: MINIFICPP-1455: Fix FileStream error handling and reporting

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



##########
File path: libminifi/test/unit/FileStreamTests.cpp
##########
@@ -263,3 +267,75 @@ TEST_CASE("Read zero bytes") {
   minifi::io::FileStream stream(utils::file::concat_path(dir, "test.txt"), 0, true);
   REQUIRE(stream.read(nullptr, 0) == 0);
 }
+
+TEST_CASE("Non-existing file read/write test") {
+  TestController test_controller;
+  char format[] = "/tmp/gt.XXXXXX";
+  auto dir = test_controller.createTempDirectory(format);
+  minifi::io::FileStream stream(utils::file::concat_path(dir, "non_existing_file.txt"), 0, true);
+  REQUIRE(test_controller.getLog().getInstance().contains("Error opening file", std::chrono::seconds(0)));
+  REQUIRE(test_controller.getLog().getInstance().contains("No such file or directory", std::chrono::seconds(0)));
+  REQUIRE(stream.write("lorem ipsum", false) == -1);
+  REQUIRE(test_controller.getLog().getInstance().contains("Error writing to file: invalid file stream", std::chrono::seconds(0)));
+  std::vector<uint8_t> readBuffer;
+  stream.seek(0);
+  REQUIRE(stream.read(readBuffer, 1) == -1);
+  REQUIRE(test_controller.getLog().getInstance().contains("Error reading from file: invalid file stream", std::chrono::seconds(0)));
+}
+
+TEST_CASE("Existing file read/write test") {
+  TestController test_controller;
+  char format[] = "/tmp/gt.XXXXXX";
+  auto dir = test_controller.createTempDirectory(format);
+  std::string path_to_existing_file(utils::file::concat_path(dir, "existing_file.txt"));
+  {
+    std::ofstream outfile(path_to_existing_file);
+    outfile << "lorem ipsum" << std::endl;
+    outfile.close();
+  }
+  minifi::io::FileStream stream(path_to_existing_file, 0, true);
+  REQUIRE_FALSE(test_controller.getLog().getInstance().contains("Error opening file", std::chrono::seconds(0)));
+  REQUIRE_FALSE(stream.write("dolor sit amet", false) == -1);
+  REQUIRE_FALSE(test_controller.getLog().getInstance().contains("Error writing to file", std::chrono::seconds(0)));
+  std::vector<uint8_t> readBuffer;
+  stream.seek(0);
+  REQUIRE_FALSE(stream.read(readBuffer, 11) == -1);
+  REQUIRE_FALSE(test_controller.getLog().getInstance().contains("Error reading from file", std::chrono::seconds(0)));
+  stream.seek(0);
+  REQUIRE(stream.read(nullptr, 11) == -1);
+  REQUIRE(test_controller.getLog().getInstance().contains("Error reading from file: invalid buffer", std::chrono::seconds(0)));
+}
+
+#ifdef USE_BOOST
+TEST_CASE("Opening file without permission creates error logs") {
+  TestController test_controller;
+  char format[] = "/tmp/gt.XXXXXX";
+  auto dir = test_controller.createTempDirectory(format);
+  std::string path_to_permissionless_file(utils::file::concat_path(dir, "permissionless_file.txt"));
+  {
+    std::ofstream outfile(path_to_permissionless_file);
+    outfile << "this file has been just created" << std::endl;
+    outfile.close();
+    // This could be done with C++17 std::filesystem
+    boost::filesystem::permissions(path_to_permissionless_file, boost::filesystem::no_perms);

Review comment:
       I feel like its useful to run this on windows as well (if there is boost support), because it verifies that the error messages provided by strerror() are the same on all platforms. (I wasn't sure about it previously given how differently windows and unix handles file permissions).
   Maybe we should do both, if there is boost use this if not and we are not on windows use the FileUtils one (which would then use chmod)?




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] lordgamez commented on a change in pull request #987: MINIFICPP-1455: Fix FileStream error handling and reporting

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



##########
File path: libminifi/test/unit/FileStreamTests.cpp
##########
@@ -263,3 +267,75 @@ TEST_CASE("Read zero bytes") {
   minifi::io::FileStream stream(utils::file::concat_path(dir, "test.txt"), 0, true);
   REQUIRE(stream.read(nullptr, 0) == 0);
 }
+
+TEST_CASE("Non-existing file read/write test") {
+  TestController test_controller;
+  char format[] = "/tmp/gt.XXXXXX";
+  auto dir = test_controller.createTempDirectory(format);
+  minifi::io::FileStream stream(utils::file::concat_path(dir, "non_existing_file.txt"), 0, true);
+  REQUIRE(test_controller.getLog().getInstance().contains("Error opening file", std::chrono::seconds(0)));
+  REQUIRE(test_controller.getLog().getInstance().contains("No such file or directory", std::chrono::seconds(0)));
+  REQUIRE(stream.write("lorem ipsum", false) == -1);
+  REQUIRE(test_controller.getLog().getInstance().contains("Error writing to file: invalid file stream", std::chrono::seconds(0)));
+  std::vector<uint8_t> readBuffer;
+  stream.seek(0);
+  REQUIRE(stream.read(readBuffer, 1) == -1);
+  REQUIRE(test_controller.getLog().getInstance().contains("Error reading from file: invalid file stream", std::chrono::seconds(0)));
+}
+
+TEST_CASE("Existing file read/write test") {
+  TestController test_controller;
+  char format[] = "/tmp/gt.XXXXXX";
+  auto dir = test_controller.createTempDirectory(format);
+  std::string path_to_existing_file(utils::file::concat_path(dir, "existing_file.txt"));
+  {
+    std::ofstream outfile(path_to_existing_file);
+    outfile << "lorem ipsum" << std::endl;
+    outfile.close();
+  }
+  minifi::io::FileStream stream(path_to_existing_file, 0, true);
+  REQUIRE_FALSE(test_controller.getLog().getInstance().contains("Error opening file", std::chrono::seconds(0)));
+  REQUIRE_FALSE(stream.write("dolor sit amet", false) == -1);
+  REQUIRE_FALSE(test_controller.getLog().getInstance().contains("Error writing to file", std::chrono::seconds(0)));
+  std::vector<uint8_t> readBuffer;
+  stream.seek(0);
+  REQUIRE_FALSE(stream.read(readBuffer, 11) == -1);
+  REQUIRE_FALSE(test_controller.getLog().getInstance().contains("Error reading from file", std::chrono::seconds(0)));
+  stream.seek(0);
+  REQUIRE(stream.read(nullptr, 11) == -1);
+  REQUIRE(test_controller.getLog().getInstance().contains("Error reading from file: invalid buffer", std::chrono::seconds(0)));
+}
+
+#ifdef USE_BOOST
+TEST_CASE("Opening file without permission creates error logs") {
+  TestController test_controller;
+  char format[] = "/tmp/gt.XXXXXX";
+  auto dir = test_controller.createTempDirectory(format);
+  std::string path_to_permissionless_file(utils::file::concat_path(dir, "permissionless_file.txt"));
+  {
+    std::ofstream outfile(path_to_permissionless_file);
+    outfile << "this file has been just created" << std::endl;
+    outfile.close();
+    // This could be done with C++17 std::filesystem
+    boost::filesystem::permissions(path_to_permissionless_file, boost::filesystem::no_perms);

Review comment:
       I think that's a good idea trying both. I just really didn't insist on running it on Windows as currently we do not have boost installed on our Windows CI runners.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] martinzink commented on a change in pull request #987: MINIFICPP-1455: Fix FileStream error handling and reporting

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



##########
File path: libminifi/src/io/FileStream.cpp
##########
@@ -31,21 +31,42 @@ namespace nifi {
 namespace minifi {
 namespace io {
 
+constexpr const char *FILE_OPENING_ERROR_MSG = "Error opening file: ";
+constexpr const char *READ_ERROR_MSG = "Error reading from file: ";
+constexpr const char *WRITE_ERROR_MSG = "Error writing to file: ";
+constexpr const char *SEEK_ERROR_MSG = "Error seeking in file: ";
+constexpr const char *INVALID_FILE_STREAM_ERROR_MSG = "invalid file stream";
+constexpr const char *TELLG_CALL_ERROR_MSG = "tellg call on file stream failed";
+constexpr const char *INVALID_BUFFER_ERROR_MSG = "invalid buffer";
+constexpr const char *FLUSH_CALL_ERROR_MSG = "flush call on file stream failed";
+constexpr const char *WRITE_CALL_ERROR_MSG = "write call on file stream failed";
+constexpr const char *EMPTY_MESSAGE_ERROR_MSG = "empty message";
+constexpr const char *SEEKG_CALL_ERROR_MSG = "seekg call on file stream failed";
+constexpr const char *SEEKP_CALL_ERROR_MSG = "seekp call on file stream failed";
+
 FileStream::FileStream(const std::string &path, bool append)
     : logger_(logging::LoggerFactory<FileStream>::getLogger()),
       path_(path),
       offset_(0) {
   file_stream_ = std::unique_ptr<std::fstream>(new std::fstream());
   if (append) {
     file_stream_->open(path.c_str(), std::fstream::in | std::fstream::out | std::fstream::app | std::fstream::binary);
-    file_stream_->seekg(0, file_stream_->end);
-    file_stream_->seekp(0, file_stream_->end);
-    std::streamoff len = file_stream_->tellg();
-    length_ = len > 0 ? gsl::narrow<size_t>(len) : 0;
-    seek(offset_);
+    if (file_stream_->is_open()) {
+      seekToEndOfFile(FILE_OPENING_ERROR_MSG);
+      auto len = file_stream_->tellg();
+      if (len < 0)

Review comment:
       fixed in da737bf




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] martinzink commented on a change in pull request #987: MINIFICPP-1455: Fix FileStream error handling and reporting

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



##########
File path: libminifi/src/io/FileStream.cpp
##########
@@ -114,17 +151,18 @@ int FileStream::read(uint8_t *buf, int buflen) {
   }
   if (!IsNullOrEmpty(buf)) {
     std::lock_guard<std::mutex> lock(file_lock_);
-    if (!file_stream_) {
+    if (file_stream_ == nullptr || !file_stream_->is_open()) {
+      logging::LOG_ERROR(logger_) << read_error_msg << invalid_file_stream_error_msg;
       return -1;
     }
     file_stream_->read(reinterpret_cast<char*>(buf), buflen);
-    if ((file_stream_->rdstate() & (file_stream_->eofbit | file_stream_->failbit)) != 0) {
+    if (file_stream_->eof() || file_stream_->fail()) {
       file_stream_->clear();
       file_stream_->seekg(0, file_stream_->end);
       file_stream_->seekp(0, file_stream_->end);

Review comment:
       done




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] martinzink commented on a change in pull request #987: MINIFICPP-1455: Fix FileStream error handling and reporting

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



##########
File path: libminifi/src/io/FileStream.cpp
##########
@@ -77,10 +101,16 @@ void FileStream::close() {
 
 void FileStream::seek(uint64_t offset) {
   std::lock_guard<std::mutex> lock(file_lock_);
+  if (file_stream_ == nullptr || !file_stream_->is_open()) {
+    logging::LOG_ERROR(logger_) << seek_error << invalid_file_stream_error_msg;
+    return;
+  }
   offset_ = gsl::narrow<size_t>(offset);
   file_stream_->clear();
-  file_stream_->seekg(offset_);
-  file_stream_->seekp(offset_);
+  if (!file_stream_->seekg(offset_))
+    logging::LOG_ERROR(logger_) << seek_error << seekg_call_error_msg;

Review comment:
       That is a good question. I felt the same at first, but since the read and write pointers should be pointing to the same position, I think it would be better to call them both regardless of errors we encounter in the first call.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org