You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by fg...@apache.org on 2022/03/08 13:54:13 UTC

[nifi-minifi-cpp] 03/05: MINIFICPP-1759 Fix linux/libc++ issues, use libc++ on ubuntu

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

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

commit 7e2dbeb34f7dbca16454107d6fc35cba77662c3b
Author: Marton Szasz <sz...@apache.org>
AuthorDate: Wed Feb 16 23:27:44 2022 +0100

    MINIFICPP-1759 Fix linux/libc++ issues, use libc++ on ubuntu
    
    - Unfixed issue: OpenCV refuses to compile with libc++/C++20
    - Also, removed Boost from the requirements, but didn't remove every trace
      of boost from the codebase, as that would be too much out of scope for
      my taste
    
    Signed-off-by: Ferenc Gerlits <fg...@gmail.com>
    
    This closes #1264
---
 .github/workflows/ci.yml                           | 10 ++-
 README.md                                          | 20 +++--
 cmake/BundledBustache.cmake                        |  6 +-
 extensions/bustache/ApplyTemplate.cpp              | 99 ++++++++++------------
 extensions/bustache/ApplyTemplate.h                | 40 ++-------
 extensions/expression-language/Expression.cpp      |  4 +-
 .../standard-processors/processors/GetFile.cpp     | 16 ++--
 .../standard-processors/processors/RouteText.cpp   |  2 +-
 libminifi/CMakeLists.txt                           |  9 --
 .../include/core/state/nodes/SchedulingNodes.h     | 22 ++---
 libminifi/include/utils/OptionalUtils.h            | 27 +++---
 .../SiteToSiteProvenanceReportingTask.cpp          |  6 +-
 libminifi/src/utils/file/FileUtils.cpp             |  2 +-
 libminifi/test/bustache-tests/CMakeLists.txt       |  2 -
 14 files changed, 104 insertions(+), 161 deletions(-)

diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml
index fec1e24..d1593da 100644
--- a/.github/workflows/ci.yml
+++ b/.github/workflows/ci.yml
@@ -22,7 +22,7 @@ jobs:
       - id: install_dependencies
         run: |
           brew update
-          brew install ossp-uuid boost flex lua ccache sqliteodbc automake autoconf
+          brew install ossp-uuid flex lua ccache sqliteodbc automake autoconf
       - id: setup_env
         name: setup enviroment
         run: |
@@ -147,15 +147,17 @@ jobs:
       - id: install_deps
         run: |
           sudo apt update
-          sudo apt install -y ccache libfl-dev libpcap-dev libboost-all-dev openjdk-8-jdk maven libusb-1.0-0-dev libpng-dev libgps-dev
+          sudo apt install -y ccache libfl-dev libpcap-dev libboost-all-dev openjdk-8-jdk maven libusb-1.0-0-dev libpng-dev libgps-dev clang-12 libc++-12-dev libc++abi-12-dev
           echo "PATH=/usr/lib/ccache:$PATH" >> $GITHUB_ENV
           echo -e "127.0.0.1\t$HOSTNAME" | sudo tee -a /etc/hosts > /dev/null
       - name: build
         run: |
           ./bootstrap.sh -e -t
           cd build
-          export CC=clang
-          export CXX=clang++
+          export CC=clang-12
+          export CXX=clang++-12
+          export CXXFLAGS="${CXXFLAGS} -stdlib=libc++"
+          export LDFLAGS="${LDFLAGS} -stdlib=libc++"
           cmake -DUSE_SHARED_LIBS= -DCMAKE_BUILD_TYPE=Release -DENABLE_MQTT=ON -DENABLE_LIBRDKAFKA=ON -DENABLE_AWS=ON -DENABLE_AZURE=ON -DFAIL_ON_WARNINGS=ON ..
           cmake --build . --parallel $(nproc)
       - name: test
diff --git a/README.md b/README.md
index 8aa1865..0ba23a7 100644
--- a/README.md
+++ b/README.md
@@ -205,12 +205,13 @@ dnf install cmake \
   automake \
   libtool \
   libuuid libuuid-devel \
-  boost-devel \
   openssl-devel \
   bzip2-devel \
   xz-devel \
   doxygen \
   zlib-devel
+# (Optional) for building OpenCV or Bustache support
+dnf install boost-devel
 # (Optional) for building Python support
 dnf install python36-devel
 # (Optional) for building Lua support
@@ -242,10 +243,12 @@ apt install cmake \
   libtool \
   libcurl4-openssl-dev \
   uuid-dev uuid \
-  libboost-all-dev libssl-dev \
+  libssl-dev \
   libbz2-dev liblzma-dev \
   doxygen \
   zlib1g-dev
+# (Optional) for building OpenCV or Bustache support
+apt install libboost-all-dev
 # (Optional) for building Python support
 apt install libpython3-dev
 # (Optional) for building Lua support
@@ -272,7 +275,6 @@ brew install cmake \
   automake \
   libtool \
   ossp-uuid \
-  boost \
   openssl \
   python \
   lua \
@@ -282,6 +284,8 @@ brew install cmake \
   zlib
 brew install curl
 brew link curl --force
+# (Optional) for building OpenCV or Bustache support
+brew install boost
 # (Optional) for building USB Camera support
 brew install libusb libpng
 # (Optional) for building docker image/running system integration tests
@@ -299,9 +303,13 @@ sudo brew install libpcap
 
 ### Bootstrapping
 
-- MiNiFi C++ offers a bootstrap script in the root of our github repo that will boot strap the cmake and build process for you without the need to install dependencies yourself. To use this process, please run the command `boostrap.sh` from the root of the MiNiFi C++ source tree.
+- MiNiFi C++ offers a bootstrap script in the root of our github repo that will bootstrap the cmake and build process for you without the need to install dependencies yourself. To use this
+  process, please run the command `bootstrap.sh` from the root of the MiNiFi C++ source tree.
 
-- Per the table, below, you will be presented with a menu guided bootstrap process. You may enable and disable extensions ( further defined below ). Once you are finished selecting the features you wish to build, enter P to continue with the process. CMAKE dependencies will be resolved for your distro. You may enter command line options -n to force yes to all prompts ( including the package installation prompts ) and -b to automatically run make once the cmake process is complete. Altern [...]
+- Per the table, below, you will be presented with a menu guided bootstrap process. You may enable and disable extensions ( further defined below ). Once you are finished selecting the features
+  you wish to build, enter P to continue with the process. CMAKE dependencies will be resolved for your distro. You may enter command line options -n to force yes to all prompts
+  (including the package installation prompts ) and -b to automatically run make once the cmake process is complete. Alternatively, you may include the package argument to bootstrap, -p,
+  which will run make package.
 
 - If you provide -b or -p to bootstrap.sh, you do not need to follow the Building section, below. If you do not provide these arguments you may skip the cmake .. section from Building.
 
@@ -357,7 +365,7 @@ sudo brew install libpcap
     Enter choice [ A - Z or 1-7 ]
   ```
 
-- Boostrap now saves state between runs. State will automatically be saved. Provide -c or --clear to clear this state. The -i option provides a guided menu install with the ability to change
+- Bootstrap now saves state between runs. State will automatically be saved. Provide -c or --clear to clear this state. The -i option provides a guided menu install with the ability to change
 advanced features.
 
 ### Building
diff --git a/cmake/BundledBustache.cmake b/cmake/BundledBustache.cmake
index 97d9c4d..5c2140a 100644
--- a/cmake/BundledBustache.cmake
+++ b/cmake/BundledBustache.cmake
@@ -16,9 +16,6 @@
 # under the License.
 
 function(use_bundled_bustache SOURCE_DIR BINARY_DIR)
-    # Find Boost
-    find_package(Boost COMPONENTS system filesystem iostreams REQUIRED)
-
     # Define byproducts
     if (WIN32)
         set(BYPRODUCT "lib/bustache.lib")
@@ -57,6 +54,5 @@ function(use_bundled_bustache SOURCE_DIR BINARY_DIR)
     set_target_properties(BUSTACHE::libbustache PROPERTIES IMPORTED_LOCATION "${BUSTACHE_LIBRARY}")
     add_dependencies(BUSTACHE::libbustache bustache-external)
     file(MAKE_DIRECTORY ${BUSTACHE_INCLUDE_DIR})
-    set_property(TARGET BUSTACHE::libbustache APPEND PROPERTY INTERFACE_INCLUDE_DIRECTORIES ${BUSTACHE_INCLUDE_DIR} ${Boost_INCLUDE_DIRS})
-    set_property(TARGET BUSTACHE::libbustache APPEND PROPERTY INTERFACE_LINK_LIBRARIES ${Boost_LIBRARIES})
+    set_property(TARGET BUSTACHE::libbustache APPEND PROPERTY INTERFACE_INCLUDE_DIRECTORIES ${BUSTACHE_INCLUDE_DIR})
 endfunction(use_bundled_bustache)
diff --git a/extensions/bustache/ApplyTemplate.cpp b/extensions/bustache/ApplyTemplate.cpp
index cb6d74c..1107f8d 100644
--- a/extensions/bustache/ApplyTemplate.cpp
+++ b/extensions/bustache/ApplyTemplate.cpp
@@ -19,81 +19,76 @@
  */
 #include "ApplyTemplate.h"
 
-#include <iostream>
+#include <filesystem>
 #include <fstream>
+#include <iostream>
 #include <memory>
-#include <set>
 #include <string>
-
-#include <boost/iostreams/device/mapped_file.hpp>
-
-#include <bustache/model.hpp>
+#include <utility>
 
 #include "core/Resource.h"
+#include "bustache/model.hpp"
+
+namespace org::apache::nifi::minifi::processors {
+
+const core::Property ApplyTemplate::Template("Template", "Path to the input mustache template file", "");
+const core::Relationship ApplyTemplate::Success("success", "success operational on the flow record");
+
+namespace {
+class WriteCallback : public OutputStreamCallback {
+ public:
+  WriteCallback(std::filesystem::path templateFile, const core::FlowFile& flow_file)
+      :template_file_{std::move(templateFile)}, flow_file_{flow_file}
+  {}
+  int64_t process(const std::shared_ptr<io::BaseStream>& stream) override {
+    logger_->log_info("ApplyTemplate reading template file from %s", template_file_);
+    // TODO(szaszm): we might want to return to memory-mapped input files when the next todo is done. Until then, the agents stores the whole result in memory anyway, so no point in not doing the same
+    // with the template file itself
+    const auto template_file_contents = [this] {
+      std::ifstream ifs{template_file_};
+      return std::string{std::istreambuf_iterator<char>{ifs}, std::istreambuf_iterator<char>{}};
+    }();
+
+    bustache::format format(template_file_contents);
+    bustache::object data;
+
+    for (const auto &attr : flow_file_.getAttributes()) {
+      data[attr.first] = attr.second;
+    }
+
+    // TODO(calebj) write ostream reciever for format() to prevent excessive copying
+    std::string ostring = to_string(format(data));
+    stream->write(gsl::make_span(ostring).as_span<const std::byte>());
+    return gsl::narrow<int64_t>(ostring.length());
+  }
 
-namespace org {
-namespace apache {
-namespace nifi {
-namespace minifi {
-namespace processors {
-
-core::Property ApplyTemplate::Template("Template", "Path to the input mustache template file", "");
-core::Relationship ApplyTemplate::Success("success", "success operational on the flow record");
+ private:
+  std::shared_ptr<core::logging::Logger> logger_ = core::logging::LoggerFactory<ApplyTemplate>::getLogger();
+  std::filesystem::path template_file_;
+  const core::FlowFile& flow_file_;
+};
+}  // namespace
 
 void ApplyTemplate::initialize() {
-  //! Set the supported properties
-  std::set<core::Property> properties;
-  properties.insert(Template);
-  setSupportedProperties(properties);
-  //! Set the supported relationships
-  std::set<core::Relationship> relationships;
-  relationships.insert(Success);
-  setSupportedRelationships(relationships);
+  setSupportedProperties({Template});
+  setSupportedRelationships({Success});
 }
 
 void ApplyTemplate::onTrigger(const std::shared_ptr<core::ProcessContext> &context,
                               const std::shared_ptr<core::ProcessSession> &session) {
   auto flow_file = session->get();
-
   if (!flow_file) {
     return;
   }
 
   std::string template_file;
   context->getProperty(Template, template_file, flow_file);
-  WriteCallback cb(template_file, flow_file);
+  WriteCallback cb(template_file, *flow_file);
   session->write(flow_file, &cb);
   session->transfer(flow_file, Success);
 }
 
-ApplyTemplate::WriteCallback::WriteCallback(const std::string &path, const std::shared_ptr<core::FlowFile> &flow_file) {
-  template_file_ = path;
-  flow_file_ = flow_file;
-}
-
-int64_t ApplyTemplate::WriteCallback::process(const std::shared_ptr<io::BaseStream>& stream) {
-  logger_->log_info("ApplyTemplate reading template file from %s", template_file_);
-  boost::iostreams::mapped_file_source file(template_file_);
-
-  bustache::format format(file);
-  bustache::object data;
-
-  for (const auto &attr : flow_file_->getAttributes()) {
-    data[attr.first] = attr.second;
-  }
-
-  // TODO(calebj) write ostream reciever for format() to prevent excessive copying
-  std::string ostring = to_string(format(data));
-  stream->write(reinterpret_cast<const uint8_t *>(ostring.c_str()), ostring.length());
-
-  return ostring.length();
-}
-
 REGISTER_RESOURCE(ApplyTemplate, "Applies the mustache template specified by the \"Template\" property and writes the output to the flow file content. "
     "FlowFile attributes are used as template parameters.");
 
-} /* namespace processors */
-} /* namespace minifi */
-} /* namespace nifi */
-} /* namespace apache */
-} /* namespace org */
+}  // namespace org::apache::nifi::minifi::processors
diff --git a/extensions/bustache/ApplyTemplate.h b/extensions/bustache/ApplyTemplate.h
index 595c2f2..5cc1d5a 100644
--- a/extensions/bustache/ApplyTemplate.h
+++ b/extensions/bustache/ApplyTemplate.h
@@ -26,53 +26,27 @@
 #include "core/ProcessSession.h"
 #include "core/FlowFile.h"
 
-namespace org {
-namespace apache {
-namespace nifi {
-namespace minifi {
-namespace processors {
+namespace org::apache::nifi::minifi::processors {
 
 /**
  * Applies a mustache template using incoming attributes as template parameters.
  */
 class ApplyTemplate : public core::Processor {
  public:
-  /*!
-   * Create a new processor
-   */
   explicit ApplyTemplate(const std::string& name, const utils::Identifier& uuid = {})
       : Processor(name, uuid) {}
-  ~ApplyTemplate() = default;
+  ~ApplyTemplate() override = default;
   static constexpr char const *ProcessorName = "ApplyTemplate";
 
-  //! Supported Properties
-  static core::Property Template;
+  static const core::Property Template;
 
-  //! Supported Relationships
-  static core::Relationship Success;
+  static const core::Relationship Success;
 
   void onTrigger(const std::shared_ptr<core::ProcessContext> &context,
-                 const std::shared_ptr<core::ProcessSession> &session);
-  void initialize(void);
-
-  //! Write callback for outputting files generated by applying template to input
-  class WriteCallback : public OutputStreamCallback {
-   public:
-    WriteCallback(const std::string &templateFile, const std::shared_ptr<core::FlowFile> &flow_file);
-    int64_t process(const std::shared_ptr<io::BaseStream>& stream);
-
-   private:
-    std::shared_ptr<core::logging::Logger> logger_ = core::logging::LoggerFactory<ApplyTemplate::WriteCallback>::getLogger();
-    std::string template_file_;
-    std::shared_ptr<core::FlowFile> flow_file_;
-  };
-
+                 const std::shared_ptr<core::ProcessSession> &session) override;
+  void initialize() override;
  private:
   std::shared_ptr<core::logging::Logger> logger_ = core::logging::LoggerFactory<ApplyTemplate>::getLogger();
 };
 
-} /* namespace processors */
-} /* namespace minifi */
-} /* namespace nifi */
-} /* namespace apache */
-} /* namespace org */
+}  // namespace org::apache::nifi::minifi::processors
diff --git a/extensions/expression-language/Expression.cpp b/extensions/expression-language/Expression.cpp
index 7ad7107..e4168a7 100644
--- a/extensions/expression-language/Expression.cpp
+++ b/extensions/expression-language/Expression.cpp
@@ -636,7 +636,7 @@ Value expr_toDate(const std::vector<Value> &args) {
   auto utc = date::locate_zone("UTC");
   auto utct = date::make_zoned(utc, t);
   auto zt = date::make_zoned(zone, utct.get_local_time());
-  return Value(std::chrono::duration_cast<std::chrono::milliseconds>(zt.get_sys_time().time_since_epoch()).count());
+  return Value(int64_t{std::chrono::duration_cast<std::chrono::milliseconds>(zt.get_sys_time().time_since_epoch()).count()});
 }
 
 #else
@@ -686,7 +686,7 @@ Value expr_toDate(const std::vector<Value>&) {
 #endif  // EXPRESSION_LANGUAGE_USE_DATE
 
 Value expr_now(const std::vector<Value>& /*args*/) {
-  return Value(std::chrono::duration_cast<std::chrono::milliseconds>(std::chrono::system_clock::now().time_since_epoch()).count());
+  return Value(int64_t{std::chrono::duration_cast<std::chrono::milliseconds>(std::chrono::system_clock::now().time_since_epoch()).count()});
 }
 
 Value expr_unescapeCsv(const std::vector<Value> &args) {
diff --git a/extensions/standard-processors/processors/GetFile.cpp b/extensions/standard-processors/processors/GetFile.cpp
index 02d9c72..e99afd1 100644
--- a/extensions/standard-processors/processors/GetFile.cpp
+++ b/extensions/standard-processors/processors/GetFile.cpp
@@ -40,11 +40,7 @@
 
 using namespace std::literals::chrono_literals;
 
-namespace org {
-namespace apache {
-namespace nifi {
-namespace minifi {
-namespace processors {
+namespace org::apache::nifi::minifi::processors {
 
 core::Property GetFile::BatchSize(
     core::PropertyBuilder::createProperty("Batch Size")->withDescription("The maximum number of files to pull in each iteration")->withDefaultValue<uint32_t>(10)->build());
@@ -133,7 +129,9 @@ void GetFile::onSchedule(core::ProcessContext *context, core::ProcessSessionFact
     core::Property::StringToInt(value, request_.minSize);
   }
 
-  context->getProperty(PollInterval.getName(), request_.pollInterval);
+  if (const auto poll_interval = context->getProperty<core::TimePeriodValue>(PollInterval)) {
+    request_.pollInterval = poll_interval->getMilliseconds();
+  }
 
   if (context->getProperty(Recurse.getName(), value)) {
     request_.recursive = org::apache::nifi::minifi::utils::StringUtils::toBool(value).value_or(true);
@@ -291,8 +289,4 @@ int16_t GetFile::getMetricNodes(std::vector<std::shared_ptr<state::response::Res
 
 REGISTER_RESOURCE(GetFile, "Creates FlowFiles from files in a directory. MiNiFi will ignore files for which it doesn't have read permissions.");
 
-}  // namespace processors
-}  // namespace minifi
-}  // namespace nifi
-}  // namespace apache
-}  // namespace org
+}  // namespace org::apache::nifi::minifi::processors
diff --git a/extensions/standard-processors/processors/RouteText.cpp b/extensions/standard-processors/processors/RouteText.cpp
index 93eef74..e1eab5c 100644
--- a/extensions/standard-processors/processors/RouteText.cpp
+++ b/extensions/standard-processors/processors/RouteText.cpp
@@ -23,7 +23,7 @@
 #include <algorithm>
 #include <set>
 
-#ifdef __APPLE__
+#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>;
diff --git a/libminifi/CMakeLists.txt b/libminifi/CMakeLists.txt
index 00eec75..367da27 100644
--- a/libminifi/CMakeLists.txt
+++ b/libminifi/CMakeLists.txt
@@ -88,15 +88,6 @@ else()
 	set_target_properties(core-minifi PROPERTIES LIBRARY_OUTPUT_DIRECTORY "${CMAKE_BINARY_DIR}/bin")
 endif()
 
-if(NOT EXCLUDE_BOOST)
-  find_package(Boost COMPONENTS system filesystem)
-  if(Boost_FOUND)
-    add_definitions(-DUSE_BOOST)
-    target_include_directories(core-minifi PRIVATE "${Boost_INCLUDE_DIRS}")
-    list(APPEND LIBMINIFI_LIBRARIES ${Boost_SYSTEM_LIBRARY} ${Boost_FILESYSTEM_LIBRARY})
-  endif()
-endif()
-
 include(RangeV3)
 list(APPEND LIBMINIFI_LIBRARIES yaml-cpp ZLIB::ZLIB concurrentqueue RapidJSON spdlog cron Threads::Threads gsl-lite libsodium range-v3 expected-lite date::date)
 if(NOT WIN32)
diff --git a/libminifi/include/core/state/nodes/SchedulingNodes.h b/libminifi/include/core/state/nodes/SchedulingNodes.h
index 45dfdbe..ba3b561 100644
--- a/libminifi/include/core/state/nodes/SchedulingNodes.h
+++ b/libminifi/include/core/state/nodes/SchedulingNodes.h
@@ -25,13 +25,7 @@
 #include "MetricsBase.h"
 #include "core/ProcessorConfig.h"
 
-namespace org {
-namespace apache {
-namespace nifi {
-namespace minifi {
-namespace state {
-namespace response {
-
+namespace org::apache::nifi::minifi::state::response {
 
 class SchedulingDefaults : public DeviceInformation {
  public:
@@ -61,13 +55,13 @@ class SchedulingDefaults : public DeviceInformation {
 
     SerializedResponseNode defaultSchedulingPeriod;
     defaultSchedulingPeriod.name = "defaultSchedulingPeriodMillis";
-    defaultSchedulingPeriod.value = core::DEFAULT_SCHEDULING_PERIOD_MILLIS.count();
+    defaultSchedulingPeriod.value = int64_t{core::DEFAULT_SCHEDULING_PERIOD_MILLIS.count()};
 
     schedulingDefaults.children.push_back(defaultSchedulingPeriod);
 
     SerializedResponseNode defaultRunDuration;
     defaultRunDuration.name = "defaultRunDurationNanos";
-    defaultRunDuration.value = core::DEFAULT_RUN_DURATION.count();
+    defaultRunDuration.value = int64_t{core::DEFAULT_RUN_DURATION.count()};
 
     schedulingDefaults.children.push_back(defaultRunDuration);
 
@@ -79,7 +73,7 @@ class SchedulingDefaults : public DeviceInformation {
 
     SerializedResponseNode yieldDuration;
     yieldDuration.name = "yieldDurationMillis";
-    yieldDuration.value = std::chrono::milliseconds(core::DEFAULT_YIELD_PERIOD_SECONDS).count();
+    yieldDuration.value = int64_t{std::chrono::milliseconds(core::DEFAULT_YIELD_PERIOD_SECONDS).count()};
 
     schedulingDefaults.children.push_back(yieldDuration);
 
@@ -95,12 +89,6 @@ class SchedulingDefaults : public DeviceInformation {
   }
 };
 
-
-}  // namespace response
-}  // namespace state
-}  // namespace minifi
-}  // namespace nifi
-}  // namespace apache
-}  // namespace org
+}  // namespace org::apache::nifi::minifi::state::response
 
 #endif  // LIBMINIFI_INCLUDE_CORE_STATE_NODES_SCHEDULINGNODES_H_
diff --git a/libminifi/include/utils/OptionalUtils.h b/libminifi/include/utils/OptionalUtils.h
index 992493e..0572616 100644
--- a/libminifi/include/utils/OptionalUtils.h
+++ b/libminifi/include/utils/OptionalUtils.h
@@ -43,22 +43,19 @@ struct is_optional<std::optional<T>, void> : std::true_type {};
 namespace detail {
 // map implementation
 template<typename SourceType, typename F>
-auto operator|(const std::optional<SourceType>& o, map_wrapper<F> f) noexcept(noexcept(std::invoke(std::forward<F>(f.function), *o)))
-    -> std::optional<typename std::decay<decltype(std::invoke(std::forward<F>(f.function), *o))>::type> {
-  if (o.has_value()) {
-    return std::make_optional(std::invoke(std::forward<F>(f.function), *o));
+auto operator|(std::optional<SourceType> o, map_wrapper<F> f) noexcept(noexcept(std::invoke(std::forward<F>(f.function), *std::move(o)))) {
+  using cb_result = std::decay_t<std::invoke_result_t<F, SourceType>>;
+  if constexpr(std::is_same_v<cb_result, void>) {
+    if (o.has_value()) {
+      std::invoke(std::forward<F>(f.function), *std::move(o));
+    }
   } else {
-    return std::nullopt;
-  }
-}
-
-template<typename SourceType, typename F>
-auto operator|(std::optional<SourceType>&& o, map_wrapper<F> f) noexcept(noexcept(std::invoke(std::forward<F>(f.function), std::move(*o))))
-    -> std::optional<typename std::decay<decltype(std::invoke(std::forward<F>(f.function), std::move(*o)))>::type> {
-  if (o.has_value()) {
-    return std::make_optional(std::invoke(std::forward<F>(f.function), std::move(*o)));
-  } else {
-    return std::nullopt;
+    using return_type = std::optional<cb_result>;
+    if (o.has_value()) {
+      return return_type{std::make_optional(std::invoke(std::forward<F>(f.function), *std::move(o)))};
+    } else {
+      return return_type{std::nullopt};
+    }
   }
 }
 
diff --git a/libminifi/src/core/reporting/SiteToSiteProvenanceReportingTask.cpp b/libminifi/src/core/reporting/SiteToSiteProvenanceReportingTask.cpp
index fea406a..85d8af0 100644
--- a/libminifi/src/core/reporting/SiteToSiteProvenanceReportingTask.cpp
+++ b/libminifi/src/core/reporting/SiteToSiteProvenanceReportingTask.cpp
@@ -113,9 +113,9 @@ void SiteToSiteProvenanceReportingTask::getJsonReport(const std::shared_ptr<core
     rapidjson::Value parentUuidJson(rapidjson::kArrayType);
     rapidjson::Value childUuidJson(rapidjson::kArrayType);
 
-    recordJson.AddMember("timestampMillis", std::chrono::duration_cast<std::chrono::milliseconds>(record->getEventTime().time_since_epoch()).count(), alloc);
-    recordJson.AddMember("durationMillis", record->getEventDuration().count(), alloc);
-    recordJson.AddMember("lineageStart", std::chrono::duration_cast<std::chrono::milliseconds>(record->getlineageStartDate().time_since_epoch()).count(), alloc);
+    recordJson.AddMember("timestampMillis", int64_t{std::chrono::duration_cast<std::chrono::milliseconds>(record->getEventTime().time_since_epoch()).count()}, alloc);
+    recordJson.AddMember("durationMillis", int64_t{record->getEventDuration().count()}, alloc);
+    recordJson.AddMember("lineageStart", int64_t{std::chrono::duration_cast<std::chrono::milliseconds>(record->getlineageStartDate().time_since_epoch()).count()}, alloc);
     recordJson.AddMember("entitySize", record->getFileSize(), alloc);
     recordJson.AddMember("entityOffset", record->getFileOffset(), alloc);
 
diff --git a/libminifi/src/utils/file/FileUtils.cpp b/libminifi/src/utils/file/FileUtils.cpp
index 0caa7a9..3fc50fa 100644
--- a/libminifi/src/utils/file/FileUtils.cpp
+++ b/libminifi/src/utils/file/FileUtils.cpp
@@ -89,7 +89,7 @@ bool contains(const std::filesystem::path& file_path, std::string_view text_to_s
 time_t to_time_t(const std::filesystem::file_time_type file_time) {
 #if defined(WIN32)
   return std::chrono::system_clock::to_time_t(std::chrono::utc_clock::to_sys(std::chrono::file_clock::to_utc(file_time)));
-#elif defined(__APPLE__)
+#elif defined(_LIBCPP_VERSION)
   return std::chrono::file_clock::to_time_t(file_time);
 #else
   return std::chrono::system_clock::to_time_t(std::chrono::file_clock::to_sys(file_time));
diff --git a/libminifi/test/bustache-tests/CMakeLists.txt b/libminifi/test/bustache-tests/CMakeLists.txt
index 7756845..ba13664 100644
--- a/libminifi/test/bustache-tests/CMakeLists.txt
+++ b/libminifi/test/bustache-tests/CMakeLists.txt
@@ -19,7 +19,6 @@
 
 file(GLOB BUSTACHE_INTEGRATION_TESTS "*.cpp")
 SET(BUSTACHE-EXTENSIONS_TEST_COUNT 0)
-find_package(Boost COMPONENTS iostreams REQUIRED)
 FOREACH(testfile ${BUSTACHE_INTEGRATION_TESTS})
   get_filename_component(testfilename "${testfile}" NAME_WE)
   add_executable("${testfilename}" "${testfile}")
@@ -32,7 +31,6 @@ FOREACH(testfile ${BUSTACHE_INTEGRATION_TESTS})
 
   createTests("${testfilename}")
   target_link_libraries(${testfilename} ${CATCH_MAIN_LIB})
-  target_link_libraries(${testfilename} ${Boost_IOSTREAMS_LIBRARY})
   MATH(EXPR BUSTACHE-EXTENSIONS_TEST_COUNT "${BUSTACHE-EXTENSIONS_TEST_COUNT}+1")
   add_test(NAME "${testfilename}" COMMAND "${testfilename}" WORKING_DIRECTORY ${TEST_DIR})
 ENDFOREACH()