You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by am...@apache.org on 2022/05/06 14:12:19 UTC

[nifi-minifi-cpp] branch main updated (107f33955 -> d45587a24)

This is an automated email from the ASF dual-hosted git repository.

amarkovics pushed a change to branch main
in repository https://gitbox.apache.org/repos/asf/nifi-minifi-cpp.git


    from 107f33955 MINIFICPP-1795 - C2ClearCoreComponentStateTest fails sometimes on CI
     new be1878175 MINIFICPP-1824 Remove custom validator from ListenSyslog Closes #1327
     new 92fb6e9ed MINIFICPP-1818 - Change searcher Closes #1323
     new d45587a24 MINIFICPP-1807 improve stat() error logging Closes #1311

The 3 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 extensions/libarchive/UnfocusArchiveEntry.cpp      |  3 +-
 .../standard-processors/processors/GetFile.cpp     | 19 +++---
 .../standard-processors/processors/ListFile.h      |  2 +-
 .../processors/ListenSyslog.cpp                    |  5 +-
 .../standard-processors/processors/RouteText.cpp   | 16 +----
 libminifi/include/utils/Searcher.h                 | 73 ++++++++++++++++++++++
 libminifi/include/utils/file/FileUtils.h           |  9 +--
 libminifi/src/utils/file/FileUtils.cpp             | 12 ++--
 8 files changed, 103 insertions(+), 36 deletions(-)
 create mode 100644 libminifi/include/utils/Searcher.h


[nifi-minifi-cpp] 03/03: MINIFICPP-1807 improve stat() error logging Closes #1311

Posted by am...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

amarkovics pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/nifi-minifi-cpp.git

commit d45587a248943496bf06310e13ebd3ad6b173dd6
Author: Marton Szasz <sz...@apache.org>
AuthorDate: Wed Apr 20 19:08:30 2022 +0200

    MINIFICPP-1807 improve stat() error logging
    Closes #1311
    
    ...and change a few filesystem utilities.
    
    - renamed utils::file::to_sys_time_point to to_sys
    - passing file_time_type by value, because it's just an integer
    - added a fix for libc++14, which behaves more like libstdc++ when it comes to file_clock
    
    Signed-off-by: Adam Markovics <nu...@gmail.com>
---
 extensions/libarchive/UnfocusArchiveEntry.cpp         |  3 ++-
 extensions/standard-processors/processors/GetFile.cpp | 19 +++++++++----------
 extensions/standard-processors/processors/ListFile.h  |  2 +-
 libminifi/include/utils/file/FileUtils.h              |  9 +++++----
 libminifi/src/utils/file/FileUtils.cpp                | 12 ++++++------
 5 files changed, 23 insertions(+), 22 deletions(-)

diff --git a/extensions/libarchive/UnfocusArchiveEntry.cpp b/extensions/libarchive/UnfocusArchiveEntry.cpp
index 2f6c1ac74..fc1f94b94 100644
--- a/extensions/libarchive/UnfocusArchiveEntry.cpp
+++ b/extensions/libarchive/UnfocusArchiveEntry.cpp
@@ -25,6 +25,7 @@
 #include <memory>
 #include <string>
 #include <set>
+#include <system_error>
 
 #include "archive.h"
 #include "archive_entry.h"
@@ -181,7 +182,7 @@ int64_t UnfocusArchiveEntry::WriteCallback::operator()(const std::shared_ptr<io:
     if (entryMetadata.entryType == AE_IFREG && entryMetadata.entrySize > 0) {
       size_t stat_ok = stat(entryMetadata.tmpFileName.c_str(), &st);
       if (stat_ok != 0) {
-        logger_->log_error("Error statting %s: %d", entryMetadata.tmpFileName, stat_ok);
+        logger_->log_error("Error statting %s: %s", entryMetadata.tmpFileName, std::system_category().default_error_condition(errno).message());
       }
       archive_entry_copy_stat(entry, &st);
     }
diff --git a/extensions/standard-processors/processors/GetFile.cpp b/extensions/standard-processors/processors/GetFile.cpp
index 2d59cdc25..49a529412 100644
--- a/extensions/standard-processors/processors/GetFile.cpp
+++ b/extensions/standard-processors/processors/GetFile.cpp
@@ -25,6 +25,7 @@
 #include <queue>
 #include <map>
 #include <memory>
+#include <regex>
 #include <set>
 #include <string>
 
@@ -231,19 +232,17 @@ std::queue<std::string> GetFile::pollListing(uint64_t batch_size) {
 bool GetFile::fileMatchesRequestCriteria(std::string fullName, std::string name, const GetFileRequest &request) {
   logger_->log_trace("Checking file: %s", fullName);
 
-#ifdef WIN32
-  struct _stat64 statbuf;
-  if (_stat64(fullName.c_str(), &statbuf) != 0) {
+  std::error_code ec;
+  uint64_t file_size = std::filesystem::file_size(fullName, ec);
+  if (ec) {
+    logger_->log_error("file_size of %s: %s", fullName, ec.message());
     return false;
   }
-#else
-  struct stat statbuf;
-  if (stat(fullName.c_str(), &statbuf) != 0) {
+  const auto modifiedTime = std::filesystem::last_write_time(fullName, ec);
+  if (ec) {
+    logger_->log_error("last_write_time of %s: %s", fullName, ec.message());
     return false;
   }
-#endif
-  uint64_t file_size = gsl::narrow<uint64_t>(statbuf.st_size);
-  auto modifiedTime = std::chrono::system_clock::time_point() + std::chrono::seconds(gsl::narrow<uint64_t>(statbuf.st_mtime));
 
   if (request.minSize > 0 && file_size < request.minSize)
     return false;
@@ -251,7 +250,7 @@ bool GetFile::fileMatchesRequestCriteria(std::string fullName, std::string name,
   if (request.maxSize > 0 && file_size > request.maxSize)
     return false;
 
-  auto fileAge = std::chrono::duration_cast<std::chrono::milliseconds>(std::chrono::system_clock::now() - modifiedTime);
+  auto fileAge = std::chrono::duration_cast<std::chrono::milliseconds>(std::chrono::file_clock::now() - modifiedTime);
   if (request.minAge > 0ms && fileAge < request.minAge)
     return false;
   if (request.maxAge > 0ms && fileAge > request.maxAge)
diff --git a/extensions/standard-processors/processors/ListFile.h b/extensions/standard-processors/processors/ListFile.h
index cef41b6b1..83b3d8606 100644
--- a/extensions/standard-processors/processors/ListFile.h
+++ b/extensions/standard-processors/processors/ListFile.h
@@ -60,7 +60,7 @@ class ListFile : public core::Processor {
  private:
   struct ListedFile : public utils::ListedObject {
     [[nodiscard]] std::chrono::time_point<std::chrono::system_clock> getLastModified() const override {
-      return std::chrono::time_point_cast<std::chrono::milliseconds>(utils::file::FileUtils::to_sys_time_point(last_modified_time));
+      return std::chrono::time_point_cast<std::chrono::milliseconds>(utils::file::FileUtils::to_sys(last_modified_time));
     }
 
     [[nodiscard]] std::string getKey() const override {
diff --git a/libminifi/include/utils/file/FileUtils.h b/libminifi/include/utils/file/FileUtils.h
index c4b1b68cb..6ba8a3654 100644
--- a/libminifi/include/utils/file/FileUtils.h
+++ b/libminifi/include/utils/file/FileUtils.h
@@ -116,8 +116,9 @@ inline char get_separator(bool /*force_posix*/ = false) {
   return '/';
 }
 #endif
-time_t to_time_t(const std::filesystem::file_time_type& time);
-std::chrono::time_point<std::chrono::system_clock> to_sys_time_point(const std::filesystem::file_time_type& time);
+time_t to_time_t(std::filesystem::file_time_type time);
+
+std::chrono::system_clock::time_point to_sys(std::filesystem::file_time_type time);
 
 inline std::string normalize_path_separators(std::string path, bool force_posix = false) {
   const auto normalize_separators = [force_posix](const char c) {
@@ -161,7 +162,7 @@ inline int64_t delete_dir(const std::string &path, bool delete_files_recursively
 }
 
 inline std::chrono::time_point<std::chrono::file_clock,
-                               std::chrono::seconds> last_write_time_point(const std::string &path) {
+    std::chrono::seconds> last_write_time_point(const std::string &path) {
   std::error_code ec;
   auto result = std::filesystem::last_write_time(path, ec);
   if (ec.value() == 0) {
@@ -170,7 +171,7 @@ inline std::chrono::time_point<std::chrono::file_clock,
   return std::chrono::time_point<std::chrono::file_clock, std::chrono::seconds>{};
 }
 
-inline const std::optional<std::filesystem::file_time_type> last_write_time(const std::string &path) {
+inline std::optional<std::filesystem::file_time_type> last_write_time(const std::string &path) {
   std::error_code ec;
   auto result = std::filesystem::last_write_time(path, ec);
   if (ec.value() == 0) {
diff --git a/libminifi/src/utils/file/FileUtils.cpp b/libminifi/src/utils/file/FileUtils.cpp
index 2dc6f41f8..33602a9fe 100644
--- a/libminifi/src/utils/file/FileUtils.cpp
+++ b/libminifi/src/utils/file/FileUtils.cpp
@@ -86,20 +86,20 @@ bool contains(const std::filesystem::path& file_path, std::string_view text_to_s
   return check_range(left.size(), left.size() + right.size());
 }
 
-time_t to_time_t(const std::filesystem::file_time_type& file_time) {
+time_t to_time_t(std::filesystem::file_time_type file_time) {
 #if defined(WIN32)
-  return std::chrono::system_clock::to_time_t(to_sys_time_point(file_time));
-#elif defined(_LIBCPP_VERSION)
+  return std::chrono::system_clock::to_time_t(to_sys(file_time));
+#elif defined(_LIBCPP_VERSION) && _LIBCPP_VERSION < 14000
   return std::chrono::file_clock::to_time_t(file_time);
 #else
-  return std::chrono::system_clock::to_time_t(to_sys_time_point(file_time));
+  return std::chrono::system_clock::to_time_t(to_sys(file_time));
 #endif
 }
 
-std::chrono::time_point<std::chrono::system_clock> to_sys_time_point(const std::filesystem::file_time_type& file_time) {
+std::chrono::time_point<std::chrono::system_clock> to_sys(std::filesystem::file_time_type file_time) {
 #if defined(WIN32)
   return std::chrono::time_point_cast<std::chrono::system_clock::duration>(file_time - std::filesystem::file_time_type::clock::now() + std::chrono::system_clock::now());
-#elif defined(_LIBCPP_VERSION)
+#elif defined(_LIBCPP_VERSION) && _LIBCPP_VERSION < 14000
   return std::chrono::system_clock::from_time_t(std::chrono::file_clock::to_time_t(file_time));
 #else
   return std::chrono::file_clock::to_sys(file_time);


[nifi-minifi-cpp] 01/03: MINIFICPP-1824 Remove custom validator from ListenSyslog Closes #1327

Posted by am...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

amarkovics pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/nifi-minifi-cpp.git

commit be1878175f935243fcb60261694a8337408bcf1c
Author: Martin Zink <ma...@apache.org>
AuthorDate: Fri May 6 08:56:23 2022 +0200

    MINIFICPP-1824 Remove custom validator from ListenSyslog
    Closes #1327
    
    Signed-off-by: Adam Markovics <nu...@gmail.com>
---
 extensions/standard-processors/processors/ListenSyslog.cpp | 5 ++++-
 1 file changed, 4 insertions(+), 1 deletion(-)

diff --git a/extensions/standard-processors/processors/ListenSyslog.cpp b/extensions/standard-processors/processors/ListenSyslog.cpp
index 61316b25c..95b9c1327 100644
--- a/extensions/standard-processors/processors/ListenSyslog.cpp
+++ b/extensions/standard-processors/processors/ListenSyslog.cpp
@@ -40,7 +40,7 @@ const core::Property ListenSyslog::ProtocolProperty(
 const core::Property ListenSyslog::MaxBatchSize(
     core::PropertyBuilder::createProperty("Max Batch Size")
         ->withDescription("The maximum number of Syslog events to process at a time.")
-        ->withDefaultValue<uint64_t>(500, std::make_shared<core::UnsignedLongValidator>("Greater or equal than 1 validator", 1))
+        ->withDefaultValue<uint64_t>(500)
         ->build());
 
 const core::Property ListenSyslog::ParseMessages(
@@ -86,6 +86,9 @@ void ListenSyslog::onSchedule(const std::shared_ptr<core::ProcessContext>& conte
   gsl_Expects(context && !server_thread_.joinable() && !server_);
 
   context->getProperty(MaxBatchSize.getName(), max_batch_size_);
+  if (max_batch_size_ < 1)
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Max Batch Size property is invalid");
+
   context->getProperty(ParseMessages.getName(), parse_messages_);
 
   uint64_t max_queue_size = 0;


[nifi-minifi-cpp] 02/03: MINIFICPP-1818 - Change searcher Closes #1323

Posted by am...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

amarkovics pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/nifi-minifi-cpp.git

commit 92fb6e9ed69d7034788786f1f8948cd9d10ff56d
Author: Adam Debreceni <ad...@apache.org>
AuthorDate: Tue May 3 14:40:55 2022 +0200

    MINIFICPP-1818 - Change searcher
    Closes #1323
    
    Signed-off-by: Adam Markovics <nu...@gmail.com>
---
 .../standard-processors/processors/RouteText.cpp   | 16 +----
 libminifi/include/utils/Searcher.h                 | 73 ++++++++++++++++++++++
 2 files changed, 76 insertions(+), 13 deletions(-)

diff --git a/extensions/standard-processors/processors/RouteText.cpp b/extensions/standard-processors/processors/RouteText.cpp
index c14632ede..5892f650a 100644
--- a/extensions/standard-processors/processors/RouteText.cpp
+++ b/extensions/standard-processors/processors/RouteText.cpp
@@ -23,16 +23,6 @@
 #include <algorithm>
 #include <set>
 
-#if __cpp_lib_boyer_moore_searcher < 201603L
-#include <experimental/functional>
-template<typename It, typename Hash, typename Eq>
-using boyer_moore_searcher = std::experimental::boyer_moore_searcher<It, Hash, Eq>;
-#else
-#include <functional>
-template<typename It, typename Hash, typename Eq>
-using boyer_moore_searcher = std::boyer_moore_searcher<It, Hash, Eq>;
-#endif
-
 #include "core/ProcessSession.h"
 #include "core/Resource.h"
 #include "io/StreamPipe.h"
@@ -44,6 +34,7 @@ using boyer_moore_searcher = std::boyer_moore_searcher<It, Hash, Eq>;
 #include "range/v3/view/cache1.hpp"
 #include "utils/ProcessorConfigUtils.h"
 #include "utils/OptionalUtils.h"
+#include "utils/Searcher.h"
 
 namespace org::apache::nifi::minifi::processors {
 
@@ -219,7 +210,6 @@ class RouteText::MatchingContext {
    private:
     CasePolicy policy_;
   };
-  using Searcher = boyer_moore_searcher<std::string::const_iterator, CaseAwareHash, CaseAwareEq>;
 
  public:
   MatchingContext(core::ProcessContext& process_context, std::shared_ptr<core::FlowFile> flow_file, CasePolicy case_policy)
@@ -255,7 +245,7 @@ class RouteText::MatchingContext {
     return (string_values_[prop.getName()] = value);
   }
 
-  const Searcher& getSearcher(const core::Property& prop) {
+  const auto& getSearcher(const core::Property& prop) {
     auto it = searcher_values_.find(prop.getName());
     if (it != searcher_values_.end()) {
       return it->second.searcher_;
@@ -286,7 +276,7 @@ class RouteText::MatchingContext {
     OwningSearcher& operator=(OwningSearcher&&) = delete;
 
     std::string str_;
-    Searcher searcher_;
+    utils::Searcher<std::string::const_iterator, CaseAwareHash, CaseAwareEq> searcher_;
   };
 
   std::map<std::string, OwningSearcher> searcher_values_;
diff --git a/libminifi/include/utils/Searcher.h b/libminifi/include/utils/Searcher.h
new file mode 100644
index 000000000..91183a0d0
--- /dev/null
+++ b/libminifi/include/utils/Searcher.h
@@ -0,0 +1,73 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#pragma once
+
+#include <version>
+#include <utility>
+#include <iterator>
+
+#ifdef _LIBCPP_VERSION
+#include <functional>
+#elif __cpp_lib_boyer_moore_searcher < 201603L
+#include <experimental/functional>
+#else
+#include <functional>
+#endif
+
+namespace org::apache::nifi::minifi::utils {
+
+// platform dependent workaround for boyer_moore_searcher
+template<typename It, typename Hash = std::hash<typename std::iterator_traits<It>::value_type>, typename BinaryPredicate = std::equal_to<>>
+class Searcher {
+ public:
+  template<typename It2>
+  std::pair<It2, It2> operator()(It2 begin, It2 end) const {
+    return impl_(std::move(begin), std::move(end));
+  }
+
+#ifdef _LIBCPP_VERSION
+
+ public:
+  Searcher(It begin, It end, Hash /*hash*/ = {}, BinaryPredicate pred = {})
+      : impl_(std::move(begin), std::move(end), std::move(pred)) {}
+
+ private:
+  // fallback to default_searcher due to libcxx bug
+  std::default_searcher<It, BinaryPredicate> impl_;
+
+#elif __cpp_lib_boyer_moore_searcher < 201603L
+
+ public:
+  Searcher(It begin, It end, Hash hash = {}, BinaryPredicate pred = {})
+      : impl_(std::move(begin), std::move(end), std::move(hash), std::move(pred)) {}
+
+ private:
+  std::experimental::boyer_moore_searcher<It, Hash, BinaryPredicate> impl_;
+
+#else
+
+ public:
+  Searcher(It begin, It end, Hash hash = {}, BinaryPredicate pred = {})
+      : impl_(std::move(begin), std::move(end), std::move(hash), std::move(pred)) {}
+
+ private:
+  std::boyer_moore_searcher<It, Hash, BinaryPredicate> impl_;
+#endif
+};
+
+}  // namespace org::apache::nifi::minifi::utils