You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by GitBox <gi...@apache.org> on 2022/03/31 11:41:43 UTC

[GitHub] [nifi-minifi-cpp] lordgamez commented on a change in pull request #1283: MINIFICPP-1760 Implement ListFile processor

lordgamez commented on a change in pull request #1283:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1283#discussion_r839508846



##########
File path: extensions/standard-processors/processors/ListFile.cpp
##########
@@ -0,0 +1,278 @@
+/**
+ * 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 "ListFile.h"
+
+#include <filesystem>
+
+#include "utils/FileReaderCallback.h"
+#include "utils/StringUtils.h"
+#include "core/Resource.h"
+
+namespace org::apache::nifi::minifi::processors {
+
+const core::Property ListFile::InputDirectory(
+    core::PropertyBuilder::createProperty("Input Directory")
+      ->withDescription("The input directory from which files to pull files")

Review comment:
       Good catch, updated in c1db097ea6391e4e3b5c355e9169b283f10a2bfb

##########
File path: extensions/standard-processors/processors/ListFile.cpp
##########
@@ -0,0 +1,278 @@
+/**
+ * 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 "ListFile.h"
+
+#include <filesystem>
+
+#include "utils/FileReaderCallback.h"
+#include "utils/StringUtils.h"
+#include "core/Resource.h"
+
+namespace org::apache::nifi::minifi::processors {
+
+const core::Property ListFile::InputDirectory(
+    core::PropertyBuilder::createProperty("Input Directory")
+      ->withDescription("The input directory from which files to pull files")
+      ->isRequired(true)
+      ->build());
+
+const core::Property ListFile::RecurseSubdirectories(
+    core::PropertyBuilder::createProperty("Recurse Subdirectories")
+      ->withDescription("Indicates whether to list files from subdirectories of the directory")
+      ->withDefaultValue(true)
+      ->isRequired(true)
+      ->build());
+
+const core::Property ListFile::FileFilter(
+    core::PropertyBuilder::createProperty("File Filter")
+      ->withDescription("Only files whose names match the given regular expression will be picked up")
+      ->build());
+
+const core::Property ListFile::PathFilter(
+    core::PropertyBuilder::createProperty("Path Filter")
+      ->withDescription("When Recurse Subdirectories is true, then only subdirectories whose path matches the given regular expression will be scanned")
+      ->build());
+
+const core::Property ListFile::MinimumFileAge(
+    core::PropertyBuilder::createProperty("Minimum File Age")
+      ->withDescription("The minimum age that a file must be in order to be pulled; any file younger than this amount of time (according to last modification date) will be ignored")
+      ->isRequired(true)
+      ->withDefaultValue<core::TimePeriodValue>("0 sec")
+      ->build());
+
+const core::Property ListFile::MaximumFileAge(
+    core::PropertyBuilder::createProperty("Maximum File Age")
+      ->withDescription("The maximum age that a file must be in order to be pulled; any file older than this amount of time (according to last modification date) will be ignored")
+      ->build());
+
+const core::Property ListFile::MinimumFileSize(
+    core::PropertyBuilder::createProperty("Minimum File Size")
+      ->withDescription("The minimum size that a file must be in order to be pulled")
+      ->isRequired(true)
+      ->withDefaultValue<core::DataSizeValue>("0 B")
+      ->build());
+
+const core::Property ListFile::MaximumFileSize(
+    core::PropertyBuilder::createProperty("Maximum File Size")
+      ->withDescription("The maximum size that a file can be in order to be pulled")
+      ->build());
+
+const core::Property ListFile::IgnoreHiddenFiles(
+    core::PropertyBuilder::createProperty("Ignore Hidden Files")
+      ->withDescription("Indicates whether or not hidden files should be ignored")
+      ->withDefaultValue(true)
+      ->isRequired(true)
+      ->build());
+
+const core::Relationship ListFile::Success("success", "All FlowFiles that are received are routed to success");

Review comment:
       Updated in c1db097ea6391e4e3b5c355e9169b283f10a2bfb

##########
File path: PROCESSORS.md
##########
@@ -995,6 +996,33 @@ In the list below, the names of required properties appear in bold. Any other pr
 |success|All files are routed to success|
 
 
+## ListFile
+
+### Description
+
+Retrieves a listing of files from the local filesystem. For each file that is listed, creates a FlowFile that represents the file so that it can be fetched in conjunction with FetchFile.
+### Properties
+
+In the list below, the names of required properties appear in bold. Any other properties (not in bold) are considered optional. The table also indicates any default values, and whether a property supports the NiFi Expression Language.
+
+| Name | Default Value | Allowable Values | Description |
+| - | - | - | - |
+|**Input Directory**|||The input directory from which files to pull files|
+|**Recurse Subdirectories**|true||Indicates whether to list files from subdirectories of the directory|
+|File Filter|||Only files whose names match the given regular expression will be picked up|
+|Path Filter|||When Recurse Subdirectories is true, then only subdirectories whose path matches the given regular expression will be scanned|
+|**Minimum File Age**|0 sec||The minimum age that a file must be in order to be pulled; any file younger than this amount of time (according to last modification date) will be ignored|
+|Maximum File Age|||The maximum age that a file must be in order to be pulled; any file older than this amount of time (according to last modification date) will be ignored|
+|**Minimum File Size**|0 B||The minimum size that a file must be in order to be pulled|
+|Maximum File Size|||The maximum size that a file can be in order to be pulled|
+|**Ignore Hidden Files**|true||Indicates whether or not hidden files should be ignored|
+### Relationships
+
+| Name | Description |
+| - | - |
+|success|All FlowFiles that are received are routed to success|
+
+

Review comment:
       Yes, that probably should be helpful update in c1db097ea6391e4e3b5c355e9169b283f10a2bfb

##########
File path: extensions/standard-processors/processors/ListFile.cpp
##########
@@ -0,0 +1,278 @@
+/**
+ * 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 "ListFile.h"
+
+#include <filesystem>
+
+#include "utils/FileReaderCallback.h"
+#include "utils/StringUtils.h"
+#include "core/Resource.h"
+
+namespace org::apache::nifi::minifi::processors {
+
+const core::Property ListFile::InputDirectory(
+    core::PropertyBuilder::createProperty("Input Directory")
+      ->withDescription("The input directory from which files to pull files")
+      ->isRequired(true)
+      ->build());
+
+const core::Property ListFile::RecurseSubdirectories(
+    core::PropertyBuilder::createProperty("Recurse Subdirectories")
+      ->withDescription("Indicates whether to list files from subdirectories of the directory")
+      ->withDefaultValue(true)
+      ->isRequired(true)
+      ->build());
+
+const core::Property ListFile::FileFilter(
+    core::PropertyBuilder::createProperty("File Filter")
+      ->withDescription("Only files whose names match the given regular expression will be picked up")
+      ->build());
+
+const core::Property ListFile::PathFilter(
+    core::PropertyBuilder::createProperty("Path Filter")
+      ->withDescription("When Recurse Subdirectories is true, then only subdirectories whose path matches the given regular expression will be scanned")
+      ->build());
+
+const core::Property ListFile::MinimumFileAge(
+    core::PropertyBuilder::createProperty("Minimum File Age")
+      ->withDescription("The minimum age that a file must be in order to be pulled; any file younger than this amount of time (according to last modification date) will be ignored")
+      ->isRequired(true)
+      ->withDefaultValue<core::TimePeriodValue>("0 sec")
+      ->build());
+
+const core::Property ListFile::MaximumFileAge(
+    core::PropertyBuilder::createProperty("Maximum File Age")
+      ->withDescription("The maximum age that a file must be in order to be pulled; any file older than this amount of time (according to last modification date) will be ignored")
+      ->build());
+
+const core::Property ListFile::MinimumFileSize(
+    core::PropertyBuilder::createProperty("Minimum File Size")
+      ->withDescription("The minimum size that a file must be in order to be pulled")
+      ->isRequired(true)
+      ->withDefaultValue<core::DataSizeValue>("0 B")
+      ->build());
+
+const core::Property ListFile::MaximumFileSize(
+    core::PropertyBuilder::createProperty("Maximum File Size")
+      ->withDescription("The maximum size that a file can be in order to be pulled")
+      ->build());
+
+const core::Property ListFile::IgnoreHiddenFiles(
+    core::PropertyBuilder::createProperty("Ignore Hidden Files")
+      ->withDescription("Indicates whether or not hidden files should be ignored")
+      ->withDefaultValue(true)
+      ->isRequired(true)
+      ->build());
+
+const core::Relationship ListFile::Success("success", "All FlowFiles that are received are routed to success");
+
+void ListFile::initialize() {
+  setSupportedProperties({
+    InputDirectory,
+    RecurseSubdirectories,
+    FileFilter,
+    PathFilter,
+    MinimumFileAge,
+    MaximumFileAge,
+    MinimumFileSize,
+    MaximumFileSize,
+    IgnoreHiddenFiles
+  });
+
+  setSupportedRelationships({
+    Success
+  });
+}
+
+void ListFile::onSchedule(const std::shared_ptr<core::ProcessContext> &context, const std::shared_ptr<core::ProcessSessionFactory> &/*sessionFactory*/) {
+  gsl_Expects(context);
+
+  auto state_manager = context->getStateManager();
+  if (state_manager == nullptr) {
+    throw Exception(PROCESSOR_EXCEPTION, "Failed to get StateManager");
+  }
+  state_manager_ = std::make_unique<minifi::utils::ListingStateManager>(state_manager);
+
+  if (!context->getProperty(InputDirectory.getName(), input_directory_) || input_directory_.empty()) {
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Input Directory property missing or invalid");
+  }
+
+  context->getProperty(RecurseSubdirectories.getName(), recurse_subdirectories_);
+  std::string value;
+  if (context->getProperty(FileFilter.getName(), value) && !value.empty()) {
+    file_filter_ = std::regex(value);
+  }
+
+  if (context->getProperty(PathFilter.getName(), value) && !value.empty()) {
+    path_filter_ = std::regex(value);
+  }
+
+  if (auto minimum_file_age = context->getProperty<core::TimePeriodValue>(MinimumFileAge)) {
+    minimum_file_age_ =  minimum_file_age->getMilliseconds();
+  }
+
+  if (auto maximum_file_age = context->getProperty<core::TimePeriodValue>(MaximumFileAge)) {
+    maximum_file_age_ =  maximum_file_age->getMilliseconds();
+  }
+
+  uint64_t int_value = 0;
+  if (context->getProperty(MinimumFileSize.getName(), value) && !value.empty() && core::Property::StringToInt(value, int_value)) {
+    minimum_file_size_ = int_value;
+  }
+
+  if (context->getProperty(MaximumFileSize.getName(), value) && !value.empty() && core::Property::StringToInt(value, int_value)) {
+    maximum_file_size_ = int_value;
+  }
+
+  context->getProperty(IgnoreHiddenFiles.getName(), ignore_hidden_files_);
+}
+
+bool ListFile::fileMatchesFilters(const ListedFile& listed_file) {
+  if (ignore_hidden_files_ && utils::file::FileUtils::is_hidden(listed_file.absolute_path)) {
+    logger_->log_debug("File '%s' is hidden so it will not be listed", listed_file.absolute_path);
+    return false;
+  }
+
+  if (file_filter_ && !std::regex_match(listed_file.filename, *file_filter_)) {
+    logger_->log_debug("File '%s' does not match file filter so it will not be listed", listed_file.absolute_path);
+    return false;
+  }
+
+  if (path_filter_ && !listed_file.relative_path.empty() && !std::regex_match(listed_file.relative_path, *path_filter_)) {
+    logger_->log_debug("Relative path '%s' does not match path filter so file '%s' it will not be listed", listed_file.relative_path, listed_file.absolute_path);

Review comment:
       Updated in c1db097ea6391e4e3b5c355e9169b283f10a2bfb

##########
File path: libminifi/include/utils/file/FileUtils.h
##########
@@ -171,6 +179,19 @@ inline const std::optional<std::filesystem::file_time_type> last_write_time(cons
   return std::nullopt;
 }
 
+inline std::optional<std::string> get_last_modified_time_formatted_string(const std::string& path, const std::string& format_string) {
+  auto last_write = last_write_time(path);
+  if (!last_write) {
+    return std::nullopt;
+  }
+  auto last_write_time_t = to_time_t(*last_write);
+  std::array<char, 128U> result;
+  if (std::strftime(result.data(), result.size(), format_string.c_str(), gmtime(&last_write_time_t)) != 0) {
+    return std::string(result.data());
+  }
+  return std::nullopt;

Review comment:
       Looks good, but unfortunately it's not easy to implement. The problem is that the file_time_type specific functions are still defined in the FileUtils like to_time_t which is used by this function so I cannot move it to TimeUtils without the need to move those functions as well to TimeUtils. The other problem is that `format_time` would also return an `std::optional` which would result in a `std::optional<std::optional<std::string>>` return value.

##########
File path: libminifi/include/utils/file/FileUtils.h
##########
@@ -204,7 +224,24 @@ inline int set_permissions(const std::string &path, const uint32_t permissions)
   std::filesystem::permissions(path, static_cast<std::filesystem::perms>(permissions), ec);
   return ec.value();
 }
-#endif
+
+inline bool get_permission_string(const std::string &path, std::string &permission_string) {

Review comment:
       Updated in c1db097ea6391e4e3b5c355e9169b283f10a2bfb

##########
File path: libminifi/include/utils/file/FileUtils.h
##########
@@ -577,6 +614,147 @@ inline std::string get_file_content(const std::string &file_name) {
 bool contains(const std::filesystem::path& file_path, std::string_view text_to_search);
 
 
+inline std::optional<std::string> get_file_owner(const std::string& file_path) {
+#ifndef WIN32
+  struct stat info;
+  if (stat(file_path.c_str(), &info) != 0) {
+    return std::nullopt;
+  }
+
+  struct passwd pw;
+  pw.pw_name = 0;
+  struct passwd *result = nullptr;
+  char localbuf[1024];
+  std::fill(localbuf, localbuf + sizeof(localbuf), 0);

Review comment:
       Updated in c1db097ea6391e4e3b5c355e9169b283f10a2bfb

##########
File path: libminifi/include/utils/file/FileUtils.h
##########
@@ -577,6 +614,147 @@ inline std::string get_file_content(const std::string &file_name) {
 bool contains(const std::filesystem::path& file_path, std::string_view text_to_search);
 
 
+inline std::optional<std::string> get_file_owner(const std::string& file_path) {
+#ifndef WIN32
+  struct stat info;
+  if (stat(file_path.c_str(), &info) != 0) {
+    return std::nullopt;
+  }
+
+  struct passwd pw;
+  pw.pw_name = 0;
+  struct passwd *result = nullptr;
+  char localbuf[1024];
+  std::fill(localbuf, localbuf + sizeof(localbuf), 0);
+  if (getpwuid_r(info.st_uid, &pw, localbuf, sizeof(localbuf), &result) != 0 || pw.pw_name == 0) {
+    return std::nullopt;
+  }
+
+  return std::string(pw.pw_name);
+#else
+  DWORD return_code = 0;
+  PSID sid_owner = NULL;
+  BOOL bool_return = TRUE;
+  LPTSTR account_name = NULL;
+  LPTSTR domain_name = NULL;
+  DWORD account_name_dword = 1;
+  DWORD domain_name_dword = 1;
+  SID_NAME_USE sid_type = SidTypeUnknown;
+  HANDLE file_handle;
+  PSECURITY_DESCRIPTOR sec_descriptor = NULL;
+
+  // Get the handle of the file object.
+  file_handle = CreateFile(
+    TEXT(file_path.c_str()),
+    GENERIC_READ,
+    FILE_SHARE_READ,
+    NULL,
+    OPEN_EXISTING,
+    FILE_ATTRIBUTE_NORMAL,
+    NULL);
+
+  // Check GetLastError for CreateFile error code.
+  if (file_handle == INVALID_HANDLE_VALUE) {
+    return std::nullopt;
+  }
+
+  // Get the owner SID of the file.
+  return_code = GetSecurityInfo(
+    file_handle,
+    SE_FILE_OBJECT,
+    OWNER_SECURITY_INFORMATION,
+    &sid_owner,
+    NULL,
+    NULL,
+    NULL,
+    &sec_descriptor);
+
+  // Check GetLastError for GetSecurityInfo error condition.
+  if (return_code != ERROR_SUCCESS) {
+    return std::nullopt;
+  }
+
+  // First call to LookupAccountSid to get the buffer sizes.
+  bool_return = LookupAccountSid(
+    NULL,
+    sid_owner,
+    account_name,
+    (LPDWORD)&account_name_dword,
+    domain_name,
+    (LPDWORD)&domain_name_dword,
+    &sid_type);
+
+  // Reallocate memory for the buffers.
+  account_name = (LPTSTR)GlobalAlloc(
+    GMEM_FIXED,
+    account_name_dword);
+
+  // Check GetLastError for GlobalAlloc error condition.
+  if (account_name == NULL) {
+    return std::nullopt;
+  }
+
+  domain_name = (LPTSTR)GlobalAlloc(
+    GMEM_FIXED,
+    domain_name_dword);

Review comment:
       Good catch, added free in c1db097ea6391e4e3b5c355e9169b283f10a2bfb

##########
File path: libminifi/include/utils/file/FileUtils.h
##########
@@ -577,6 +614,147 @@ inline std::string get_file_content(const std::string &file_name) {
 bool contains(const std::filesystem::path& file_path, std::string_view text_to_search);
 
 
+inline std::optional<std::string> get_file_owner(const std::string& file_path) {
+#ifndef WIN32
+  struct stat info;
+  if (stat(file_path.c_str(), &info) != 0) {
+    return std::nullopt;
+  }
+
+  struct passwd pw;
+  pw.pw_name = 0;
+  struct passwd *result = nullptr;
+  char localbuf[1024];
+  std::fill(localbuf, localbuf + sizeof(localbuf), 0);
+  if (getpwuid_r(info.st_uid, &pw, localbuf, sizeof(localbuf), &result) != 0 || pw.pw_name == 0) {
+    return std::nullopt;
+  }
+
+  return std::string(pw.pw_name);
+#else
+  DWORD return_code = 0;
+  PSID sid_owner = NULL;
+  BOOL bool_return = TRUE;
+  LPTSTR account_name = NULL;
+  LPTSTR domain_name = NULL;
+  DWORD account_name_dword = 1;
+  DWORD domain_name_dword = 1;
+  SID_NAME_USE sid_type = SidTypeUnknown;
+  HANDLE file_handle;
+  PSECURITY_DESCRIPTOR sec_descriptor = NULL;
+
+  // Get the handle of the file object.
+  file_handle = CreateFile(
+    TEXT(file_path.c_str()),
+    GENERIC_READ,
+    FILE_SHARE_READ,
+    NULL,
+    OPEN_EXISTING,
+    FILE_ATTRIBUTE_NORMAL,
+    NULL);
+
+  // Check GetLastError for CreateFile error code.
+  if (file_handle == INVALID_HANDLE_VALUE) {
+    return std::nullopt;
+  }
+
+  // Get the owner SID of the file.
+  return_code = GetSecurityInfo(
+    file_handle,
+    SE_FILE_OBJECT,
+    OWNER_SECURITY_INFORMATION,
+    &sid_owner,
+    NULL,
+    NULL,
+    NULL,
+    &sec_descriptor);
+
+  // Check GetLastError for GetSecurityInfo error condition.
+  if (return_code != ERROR_SUCCESS) {
+    return std::nullopt;
+  }
+
+  // First call to LookupAccountSid to get the buffer sizes.
+  bool_return = LookupAccountSid(
+    NULL,
+    sid_owner,
+    account_name,
+    (LPDWORD)&account_name_dword,
+    domain_name,
+    (LPDWORD)&domain_name_dword,
+    &sid_type);
+
+  // Reallocate memory for the buffers.
+  account_name = (LPTSTR)GlobalAlloc(
+    GMEM_FIXED,
+    account_name_dword);
+
+  // Check GetLastError for GlobalAlloc error condition.
+  if (account_name == NULL) {
+    return std::nullopt;
+  }
+
+  domain_name = (LPTSTR)GlobalAlloc(
+    GMEM_FIXED,
+    domain_name_dword);
+
+  // Check GetLastError for GlobalAlloc error condition.
+  if (domain_name == NULL) {
+    return std::nullopt;
+  }
+
+  // Second call to LookupAccountSid to get the account name.
+  bool_return = LookupAccountSid(
+    NULL,                   // name of local or remote computer
+    sid_owner,              // security identifier
+    account_name,               // account name buffer
+    (LPDWORD)&account_name_dword,   // size of account name buffer
+    domain_name,             // domain name
+    (LPDWORD)&domain_name_dword,  // size of domain name buffer
+    &sid_type);                 // SID type
+
+  // Check GetLastError for LookupAccountSid error condition.
+  if (bool_return == FALSE) {
+    return std::nullopt;
+  }
+
+  return std::string(account_name);
+#endif
+}
+
+#ifndef WIN32
+inline std::optional<std::string> get_file_group(const std::string& file_path) {
+  struct stat info;
+  if (stat(file_path.c_str(), &info) != 0) {
+    return std::nullopt;
+  }
+
+  struct group gr;
+  gr.gr_name = 0;
+  struct group *result = nullptr;
+  char localbuf[1024];
+  std::fill(localbuf, localbuf + sizeof(localbuf), 0);
+  if ((getgrgid_r(info.st_uid, &gr, localbuf, sizeof(localbuf), &result) != 0) || gr.gr_name == 0) {
+    return std::nullopt;
+  }
+
+  return std::string(gr.gr_name);
+}
+#endif
+
+inline std::optional<std::string> get_relative_path(const std::string& path, const std::string& base_path) {
+  if (!utils::StringUtils::startsWith(path, base_path)) {
+    return std::nullopt;
+  }
+
+  auto relative_path = path.substr(base_path.size());
+  if (utils::StringUtils::startsWith(relative_path, std::string(1, utils::file::FileUtils::get_separator()))) {
+    return relative_path.substr(1);
+  }

Review comment:
       Updated in c1db097ea6391e4e3b5c355e9169b283f10a2bfb

##########
File path: extensions/standard-processors/processors/ListFile.cpp
##########
@@ -0,0 +1,278 @@
+/**
+ * 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 "ListFile.h"
+
+#include <filesystem>
+
+#include "utils/FileReaderCallback.h"
+#include "utils/StringUtils.h"
+#include "core/Resource.h"
+
+namespace org::apache::nifi::minifi::processors {
+
+const core::Property ListFile::InputDirectory(
+    core::PropertyBuilder::createProperty("Input Directory")
+      ->withDescription("The input directory from which files to pull files")
+      ->isRequired(true)
+      ->build());
+
+const core::Property ListFile::RecurseSubdirectories(
+    core::PropertyBuilder::createProperty("Recurse Subdirectories")
+      ->withDescription("Indicates whether to list files from subdirectories of the directory")
+      ->withDefaultValue(true)
+      ->isRequired(true)
+      ->build());
+
+const core::Property ListFile::FileFilter(
+    core::PropertyBuilder::createProperty("File Filter")
+      ->withDescription("Only files whose names match the given regular expression will be picked up")
+      ->build());
+
+const core::Property ListFile::PathFilter(
+    core::PropertyBuilder::createProperty("Path Filter")
+      ->withDescription("When Recurse Subdirectories is true, then only subdirectories whose path matches the given regular expression will be scanned")
+      ->build());
+
+const core::Property ListFile::MinimumFileAge(
+    core::PropertyBuilder::createProperty("Minimum File Age")
+      ->withDescription("The minimum age that a file must be in order to be pulled; any file younger than this amount of time (according to last modification date) will be ignored")
+      ->isRequired(true)
+      ->withDefaultValue<core::TimePeriodValue>("0 sec")
+      ->build());
+
+const core::Property ListFile::MaximumFileAge(
+    core::PropertyBuilder::createProperty("Maximum File Age")
+      ->withDescription("The maximum age that a file must be in order to be pulled; any file older than this amount of time (according to last modification date) will be ignored")
+      ->build());
+
+const core::Property ListFile::MinimumFileSize(
+    core::PropertyBuilder::createProperty("Minimum File Size")
+      ->withDescription("The minimum size that a file must be in order to be pulled")
+      ->isRequired(true)
+      ->withDefaultValue<core::DataSizeValue>("0 B")
+      ->build());
+
+const core::Property ListFile::MaximumFileSize(
+    core::PropertyBuilder::createProperty("Maximum File Size")
+      ->withDescription("The maximum size that a file can be in order to be pulled")
+      ->build());
+
+const core::Property ListFile::IgnoreHiddenFiles(
+    core::PropertyBuilder::createProperty("Ignore Hidden Files")
+      ->withDescription("Indicates whether or not hidden files should be ignored")
+      ->withDefaultValue(true)
+      ->isRequired(true)
+      ->build());
+
+const core::Relationship ListFile::Success("success", "All FlowFiles that are received are routed to success");
+
+void ListFile::initialize() {
+  setSupportedProperties({
+    InputDirectory,
+    RecurseSubdirectories,
+    FileFilter,
+    PathFilter,
+    MinimumFileAge,
+    MaximumFileAge,
+    MinimumFileSize,
+    MaximumFileSize,
+    IgnoreHiddenFiles
+  });
+
+  setSupportedRelationships({
+    Success
+  });
+}
+
+void ListFile::onSchedule(const std::shared_ptr<core::ProcessContext> &context, const std::shared_ptr<core::ProcessSessionFactory> &/*sessionFactory*/) {
+  gsl_Expects(context);
+
+  auto state_manager = context->getStateManager();
+  if (state_manager == nullptr) {
+    throw Exception(PROCESSOR_EXCEPTION, "Failed to get StateManager");
+  }
+  state_manager_ = std::make_unique<minifi::utils::ListingStateManager>(state_manager);
+
+  if (!context->getProperty(InputDirectory.getName(), input_directory_) || input_directory_.empty()) {
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Input Directory property missing or invalid");
+  }
+
+  context->getProperty(RecurseSubdirectories.getName(), recurse_subdirectories_);
+  std::string value;
+  if (context->getProperty(FileFilter.getName(), value) && !value.empty()) {
+    file_filter_ = std::regex(value);
+  }
+
+  if (context->getProperty(PathFilter.getName(), value) && !value.empty()) {
+    path_filter_ = std::regex(value);
+  }
+
+  if (auto minimum_file_age = context->getProperty<core::TimePeriodValue>(MinimumFileAge)) {
+    minimum_file_age_ =  minimum_file_age->getMilliseconds();
+  }
+
+  if (auto maximum_file_age = context->getProperty<core::TimePeriodValue>(MaximumFileAge)) {
+    maximum_file_age_ =  maximum_file_age->getMilliseconds();
+  }
+
+  uint64_t int_value = 0;
+  if (context->getProperty(MinimumFileSize.getName(), value) && !value.empty() && core::Property::StringToInt(value, int_value)) {
+    minimum_file_size_ = int_value;
+  }
+
+  if (context->getProperty(MaximumFileSize.getName(), value) && !value.empty() && core::Property::StringToInt(value, int_value)) {
+    maximum_file_size_ = int_value;
+  }
+
+  context->getProperty(IgnoreHiddenFiles.getName(), ignore_hidden_files_);
+}
+
+bool ListFile::fileMatchesFilters(const ListedFile& listed_file) {
+  if (ignore_hidden_files_ && utils::file::FileUtils::is_hidden(listed_file.absolute_path)) {
+    logger_->log_debug("File '%s' is hidden so it will not be listed", listed_file.absolute_path);
+    return false;
+  }
+
+  if (file_filter_ && !std::regex_match(listed_file.filename, *file_filter_)) {
+    logger_->log_debug("File '%s' does not match file filter so it will not be listed", listed_file.absolute_path);
+    return false;
+  }
+
+  if (path_filter_ && !listed_file.relative_path.empty() && !std::regex_match(listed_file.relative_path, *path_filter_)) {
+    logger_->log_debug("Relative path '%s' does not match path filter so file '%s' it will not be listed", listed_file.relative_path, listed_file.absolute_path);
+    return false;
+  }
+
+  auto file_age = std::chrono::duration_cast<std::chrono::milliseconds>(std::chrono::system_clock::now() - listed_file.getLastModified());
+  if (minimum_file_age_ && file_age < *minimum_file_age_) {
+    logger_->log_debug("File '%s' does not meet the minimum file age requirement so it will not be listed", listed_file.absolute_path);
+    return false;
+  }
+
+  if (maximum_file_age_ && file_age > *maximum_file_age_) {
+    logger_->log_debug("File '%s' does not meet the maximum file age requirement so it will not be listed", listed_file.absolute_path);
+    return false;
+  }
+
+  if (minimum_file_size_ && listed_file.file_size < *minimum_file_size_) {
+    logger_->log_debug("File '%s' does not meet the minimum file size requirement so it will not be listed", listed_file.absolute_path);
+    return false;
+  }
+
+  if (maximum_file_size_ && *maximum_file_size_ < listed_file.file_size) {
+    logger_->log_debug("File '%s' does not meet the maximum file size requirement so it will not be listed", listed_file.absolute_path);
+    return false;
+  }
+
+  return true;
+}
+
+std::shared_ptr<core::FlowFile> ListFile::createFlowFile(core::ProcessSession& session, const ListedFile& listed_file) {
+  auto flow_file = session.create();
+  session.putAttribute(flow_file, core::SpecialFlowAttribute::FILENAME, listed_file.filename);
+  session.putAttribute(flow_file, core::SpecialFlowAttribute::ABSOLUTE_PATH, listed_file.absolute_path);
+  session.putAttribute(flow_file, core::SpecialFlowAttribute::PATH, listed_file.relative_path.empty() ? std::string(".") + utils::file::FileUtils::get_separator() : listed_file.relative_path);
+  session.putAttribute(flow_file, "file.size", std::to_string(listed_file.file_size));
+  if (auto last_modified_str = utils::file::FileUtils::get_last_modified_time_formatted_string(listed_file.absolute_path, "%Y-%m-%dT%H:%M:%SZ")) {
+    session.putAttribute(flow_file, "file.lastModifiedTime", *last_modified_str);
+  } else {
+    session.putAttribute(flow_file, "file.lastModifiedTime", "");
+    logger_->log_warn("Could not get last modification time of file '%s'", listed_file.absolute_path);
+  }
+
+  std::string permission_string;
+  if (utils::file::FileUtils::get_permission_string(listed_file.absolute_path, permission_string)) {
+    session.putAttribute(flow_file, "file.permissions", permission_string);
+  } else {
+    logger_->log_warn("Failed to get permissions of file '%s'", listed_file.absolute_path);
+    session.putAttribute(flow_file, "file.permissions", "");
+  }
+
+  if (auto owner = utils::file::FileUtils::get_file_owner(listed_file.absolute_path)) {
+    session.putAttribute(flow_file, "file.owner", *owner);
+  } else {
+    logger_->log_warn("Failed to get owner of file '%s'", listed_file.absolute_path);
+    session.putAttribute(flow_file, "file.owner", "");
+  }
+
+#ifndef WIN32
+  if (auto group = utils::file::FileUtils::get_file_group(listed_file.absolute_path)) {
+    session.putAttribute(flow_file, "file.group", *group);
+  } else {
+    logger_->log_warn("Failed to get group of file '%s'", listed_file.absolute_path);
+    session.putAttribute(flow_file, "file.group", "");
+  }
+#else
+  session.putAttribute(flow_file, "file.group", "");

Review comment:
       In NiFi the Java filesystem API is platform independent and because of this I'm not sure what would that return. I was thinking about this before and the reason I decided to keep it as an empty value, to have all attributes present on all platforms that can be expected by this processor and also because other attributes like `file.permissions` or `file.lastModifiedTime` are also present as empty values if they cannot be retrieved, so this attribute can be consistent with that behavior.




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