You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by sz...@apache.org on 2022/04/26 14:39:27 UTC

[nifi-minifi-cpp] branch main updated (113a92970 -> fbf2a349b)

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

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


    from 113a92970 MINIFICPP-1805 Remove unused bootstrap extension
     new 1a521323e MINIFICPP-1802 Do not make EC2 HTTP calls when AWS extension is not used
     new b0f0873d7 MINIFICPP-1675 Use regex.h instead of std::regex when using libstdc++
     new fbf2a349b MINIFICPP-1810 Provide logs to be read from docker logs

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:
 .dockerignore                                      |   3 +-
 bin/minifi.sh                                      |   5 +-
 docker/.dockerignore                               |  17 --
 docker/Dockerfile                                  |   1 +
 {conf => docker/conf}/minifi-log.properties        |  14 +-
 docker/test/integration/minifi/core/Container.py   |   3 -
 .../integration/minifi/core/DockerTestCluster.py   |  22 +-
 .../integration/minifi/core/MinifiContainer.py     |   3 -
 .../test/integration/minifi/core/NifiContainer.py  |   3 -
 encrypt-config/tests/ConfigFileEncryptorTests.cpp  |   6 +-
 extensions/aws/processors/DeleteS3Object.cpp       |   3 +-
 extensions/aws/processors/FetchS3Object.cpp        |   3 +-
 extensions/aws/processors/ListS3.cpp               |   4 +-
 extensions/aws/processors/PutS3Object.cpp          |   3 +-
 extensions/aws/processors/S3Processor.cpp          |   7 +-
 extensions/aws/processors/S3Processor.h            |   2 +-
 extensions/aws/s3/S3Wrapper.cpp                    |   8 +-
 .../azure/processors/ListAzureDataLakeStorage.cpp  |   6 +-
 extensions/azure/storage/AzureDataLakeStorage.cpp  |  10 +-
 extensions/azure/storage/DataLakeStorageClient.h   |   6 +-
 extensions/civetweb/processors/ListenHTTP.cpp      |   4 +-
 extensions/civetweb/processors/ListenHTTP.h        |   6 +-
 extensions/expression-language/Expression.cpp      |  17 +-
 extensions/http-curl/client/HTTPClient.cpp         |   8 +-
 .../KubernetesControllerService.cpp                |  10 +-
 .../KubernetesControllerService.h                  |   7 +-
 extensions/librdkafka/PublishKafka.cpp             |   8 +-
 extensions/librdkafka/PublishKafka.h               |   4 +-
 extensions/pcap/CapturePacket.cpp                  |   8 +-
 extensions/sftp/processors/ListSFTP.cpp            |  13 +-
 extensions/sftp/processors/ListSFTP.h              |   6 +-
 .../processors/AttributesToJSON.cpp                |   4 +-
 .../processors/AttributesToJSON.h                  |   4 +-
 .../processors/DefragmentText.cpp                  |   6 +-
 .../processors/DefragmentText.h                    |   4 +-
 .../standard-processors/processors/ExtractText.cpp |  14 +-
 .../standard-processors/processors/GetFile.cpp     |   6 +-
 .../standard-processors/processors/ListenSyslog.h  |   1 +
 .../standard-processors/processors/RouteText.cpp   |  25 +-
 .../standard-processors/processors/RouteText.h     |   4 +-
 .../standard-processors/processors/TailFile.cpp    |  18 +-
 .../tests/unit/ExtractTextTests.cpp                |  32 +++
 .../tests/unit/RouteTextTests.cpp                  |   6 +-
 .../tests/unit/TailFileTests.cpp                   |  10 +-
 .../windows-event-log/wel/MetadataWalker.cpp       |   4 +-
 extensions/windows-event-log/wel/MetadataWalker.h  |   4 +-
 libminifi/include/utils/HTTPUtils.h                |   8 +-
 libminifi/include/utils/RegexUtils.h               | 173 ++++++++++++
 libminifi/include/utils/StringUtils.h              |   9 -
 libminifi/src/core/yaml/YamlConfiguration.cpp      |  10 +-
 libminifi/src/utils/RegexUtils.cpp                 | 295 +++++++++++++++++++++
 libminifi/src/utils/StringUtils.cpp                |  10 -
 libminifi/test/unit/RegexUtilsTests.cpp            | 129 +++++++++
 libminifi/test/unit/StringUtilsTests.cpp           |  23 --
 54 files changed, 787 insertions(+), 232 deletions(-)
 delete mode 100644 docker/.dockerignore
 copy {conf => docker/conf}/minifi-log.properties (89%)
 create mode 100644 libminifi/include/utils/RegexUtils.h
 create mode 100644 libminifi/src/utils/RegexUtils.cpp
 create mode 100644 libminifi/test/unit/RegexUtilsTests.cpp


[nifi-minifi-cpp] 02/03: MINIFICPP-1675 Use regex.h instead of std::regex when using libstdc++

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

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

commit b0f0873d74ea7a19d4c2cda7ed79a99ea46bd88d
Author: Gabor Gyimesi <ga...@gmail.com>
AuthorDate: Tue Apr 26 16:04:05 2022 +0200

    MINIFICPP-1675 Use regex.h instead of std::regex when using libstdc++
    
    Due to a bug in the libstdc++ implementation of std::regex it is unsafe
    to use std::regex_match and std::regex_search on large texts as the
    process can crash due to a stack overflow. In this commit std::regex
    usages are replaced with minifi::utils::Regex which uses regex.h if
    libstdc++ is used otherwise it uses std::regex.
    
    There are a few exceptions:
    - std::regex is still used for regex_replace calls
    - AppendHostInfo still uses std::regex as POSIX regex grammar is not
      sufficient for easily filtering interface names, and there is no risk
      of running into the bug as the interface names are small
    - In tests and mocks we still use std::regex as they are only run in our
      test environments with controlled inputs and sometimes use regex
      patterns not available in POSIX grammar
    
    Closes #1300
    Signed-off-by: Marton Szasz <sz...@apache.org>
---
 encrypt-config/tests/ConfigFileEncryptorTests.cpp  |   6 +-
 extensions/aws/s3/S3Wrapper.cpp                    |   8 +-
 .../azure/processors/ListAzureDataLakeStorage.cpp  |   6 +-
 extensions/azure/storage/AzureDataLakeStorage.cpp  |  10 +-
 extensions/azure/storage/DataLakeStorageClient.h   |   6 +-
 extensions/civetweb/processors/ListenHTTP.cpp      |   4 +-
 extensions/civetweb/processors/ListenHTTP.h        |   6 +-
 extensions/expression-language/Expression.cpp      |  17 +-
 extensions/http-curl/client/HTTPClient.cpp         |   8 +-
 .../KubernetesControllerService.cpp                |  10 +-
 .../KubernetesControllerService.h                  |   7 +-
 extensions/librdkafka/PublishKafka.cpp             |   8 +-
 extensions/librdkafka/PublishKafka.h               |   4 +-
 extensions/pcap/CapturePacket.cpp                  |   8 +-
 extensions/sftp/processors/ListSFTP.cpp            |  13 +-
 extensions/sftp/processors/ListSFTP.h              |   6 +-
 .../processors/AttributesToJSON.cpp                |   4 +-
 .../processors/AttributesToJSON.h                  |   4 +-
 .../processors/DefragmentText.cpp                  |   6 +-
 .../processors/DefragmentText.h                    |   4 +-
 .../standard-processors/processors/ExtractText.cpp |  14 +-
 .../standard-processors/processors/GetFile.cpp     |   6 +-
 .../standard-processors/processors/ListenSyslog.h  |   1 +
 .../standard-processors/processors/RouteText.cpp   |  25 +-
 .../standard-processors/processors/RouteText.h     |   4 +-
 .../standard-processors/processors/TailFile.cpp    |  18 +-
 .../tests/unit/ExtractTextTests.cpp                |  32 +++
 .../tests/unit/RouteTextTests.cpp                  |   6 +-
 .../tests/unit/TailFileTests.cpp                   |  10 +-
 .../windows-event-log/wel/MetadataWalker.cpp       |   4 +-
 extensions/windows-event-log/wel/MetadataWalker.h  |   4 +-
 libminifi/include/utils/HTTPUtils.h                |   8 +-
 libminifi/include/utils/RegexUtils.h               | 173 ++++++++++++
 libminifi/include/utils/StringUtils.h              |   9 -
 libminifi/src/core/yaml/YamlConfiguration.cpp      |  10 +-
 libminifi/src/utils/RegexUtils.cpp                 | 295 +++++++++++++++++++++
 libminifi/src/utils/StringUtils.cpp                |  10 -
 libminifi/test/unit/RegexUtilsTests.cpp            | 129 +++++++++
 libminifi/test/unit/StringUtilsTests.cpp           |  23 --
 39 files changed, 758 insertions(+), 168 deletions(-)

diff --git a/encrypt-config/tests/ConfigFileEncryptorTests.cpp b/encrypt-config/tests/ConfigFileEncryptorTests.cpp
index 32894da4a..61439ed73 100644
--- a/encrypt-config/tests/ConfigFileEncryptorTests.cpp
+++ b/encrypt-config/tests/ConfigFileEncryptorTests.cpp
@@ -17,11 +17,11 @@
 
 #include <fstream>
 #include <optional>
-#include <regex>
 #include <string>
 
 #include "ConfigFileEncryptor.h"
 #include "properties/Configuration.h"
+#include "utils/RegexUtils.h"
 
 #include "TestBase.h"
 #include "Catch.h"
@@ -46,8 +46,8 @@ bool check_encryption(const ConfigFile& test_file, const std::string& property_n
   auto length = base64_length(utils::crypto::EncryptionType::nonceLength()) +
       utils::crypto::EncryptionType::separator().size() +
       base64_length(original_value_length + utils::crypto::EncryptionType::macLength());
-  std::regex pattern("[0-9A-Za-z/+=|]{" + std::to_string(length) + "}");
-  return std::regex_match(*encrypted_value, pattern);
+  utils::Regex pattern("[0-9A-Za-z/+=|]{" + std::to_string(length) + "}");
+  return utils::regexMatch(*encrypted_value, pattern);
 }
 }  // namespace
 
diff --git a/extensions/aws/s3/S3Wrapper.cpp b/extensions/aws/s3/S3Wrapper.cpp
index fc6fa9a9e..757bf0210 100644
--- a/extensions/aws/s3/S3Wrapper.cpp
+++ b/extensions/aws/s3/S3Wrapper.cpp
@@ -20,7 +20,6 @@
 #include "S3Wrapper.h"
 
 #include <memory>
-#include <regex>
 #include <utility>
 #include <vector>
 
@@ -28,6 +27,7 @@
 #include "utils/StringUtils.h"
 #include "utils/file/FileUtils.h"
 #include "utils/gsl.h"
+#include "utils/RegexUtils.h"
 
 namespace org {
 namespace apache {
@@ -56,9 +56,9 @@ void S3Wrapper::setCannedAcl(Aws::S3::Model::PutObjectRequest& request, const st
 }
 
 Expiration S3Wrapper::getExpiration(const std::string& expiration) {
-  std::regex expr("expiry-date=\"(.*)\", rule-id=\"(.*)\"");
-  std::smatch matches;
-  const bool matched = std::regex_search(expiration, matches, expr);
+  minifi::utils::Regex expr("expiry-date=\"(.*)\", rule-id=\"(.*)\"");
+  minifi::utils::SMatch matches;
+  const bool matched = minifi::utils::regexSearch(expiration, matches, expr);
   if (!matched || matches.size() < 3)
     return Expiration{};
   return Expiration{matches[1], matches[2]};
diff --git a/extensions/azure/processors/ListAzureDataLakeStorage.cpp b/extensions/azure/processors/ListAzureDataLakeStorage.cpp
index 187dbf38c..685a93f63 100644
--- a/extensions/azure/processors/ListAzureDataLakeStorage.cpp
+++ b/extensions/azure/processors/ListAzureDataLakeStorage.cpp
@@ -112,16 +112,16 @@ std::optional<storage::ListAzureDataLakeStorageParameters> ListAzureDataLakeStor
     return std::nullopt;
   }
 
-  auto createFilterRegex = [&context](const std::string& property_name) -> std::optional<std::regex> {
+  auto createFilterRegex = [&context](const std::string& property_name) -> std::optional<minifi::utils::Regex> {
     try {
       std::string filter_str;
       context.getProperty(property_name, filter_str);
       if (!filter_str.empty()) {
-        return std::regex(filter_str);
+        return minifi::utils::Regex(filter_str);
       }
 
       return std::nullopt;
-    } catch (const std::regex_error&) {
+    } catch (const minifi::Exception&) {
       throw Exception(PROCESS_SCHEDULE_EXCEPTION, property_name + " regex is invalid");
     }
   };
diff --git a/extensions/azure/storage/AzureDataLakeStorage.cpp b/extensions/azure/storage/AzureDataLakeStorage.cpp
index 9c0ce6345..0ba2aeb73 100644
--- a/extensions/azure/storage/AzureDataLakeStorage.cpp
+++ b/extensions/azure/storage/AzureDataLakeStorage.cpp
@@ -20,7 +20,6 @@
 
 #include "AzureDataLakeStorage.h"
 
-#include <regex>
 #include <string_view>
 
 #include "AzureDataLakeStorageClient.h"
@@ -29,11 +28,12 @@
 #include "utils/StringUtils.h"
 #include "utils/gsl.h"
 #include "utils/GeneralUtils.h"
+#include "utils/RegexUtils.h"
 
 namespace org::apache::nifi::minifi::azure::storage {
 
 namespace {
-bool matchesPathFilter(std::string_view base_directory, const std::optional<std::regex>& path_regex, std::string path) {
+bool matchesPathFilter(std::string_view base_directory, const std::optional<minifi::utils::Regex>& path_regex, std::string path) {
   gsl_Expects(utils::implies(!base_directory.empty(), minifi::utils::StringUtils::startsWith(path, base_directory)));
   if (!path_regex) {
     return true;
@@ -43,15 +43,15 @@ bool matchesPathFilter(std::string_view base_directory, const std::optional<std:
     path = path.size() == base_directory.size() ? "" : path.substr(base_directory.size() + 1);
   }
 
-  return std::regex_match(path, *path_regex);
+  return minifi::utils::regexMatch(path, *path_regex);
 }
 
-bool matchesFileFilter(const std::optional<std::regex>& file_regex, const std::string& filename) {
+bool matchesFileFilter(const std::optional<minifi::utils::Regex>& file_regex, const std::string& filename) {
   if (!file_regex) {
     return true;
   }
 
-  return std::regex_match(filename, *file_regex);
+  return minifi::utils::regexMatch(filename, *file_regex);
 }
 }  // namespace
 
diff --git a/extensions/azure/storage/DataLakeStorageClient.h b/extensions/azure/storage/DataLakeStorageClient.h
index a02bfca8f..38a0a07f7 100644
--- a/extensions/azure/storage/DataLakeStorageClient.h
+++ b/extensions/azure/storage/DataLakeStorageClient.h
@@ -23,7 +23,6 @@
 #include <optional>
 #include <memory>
 #include <vector>
-#include <regex>
 
 #include "AzureStorageCredentials.h"
 
@@ -31,6 +30,7 @@
 #include "io/InputStream.h"
 #include "azure/storage/files/datalake/protocol/datalake_rest_client.hpp"
 #include "utils/Enum.h"
+#include "utils/RegexUtils.h"
 
 namespace org::apache::nifi::minifi::azure::storage {
 
@@ -63,8 +63,8 @@ struct FetchAzureDataLakeStorageParameters : public AzureDataLakeStorageFileOper
 
 struct ListAzureDataLakeStorageParameters : public AzureDataLakeStorageParameters {
   bool recurse_subdirectories = true;
-  std::optional<std::regex> path_regex;
-  std::optional<std::regex> file_regex;
+  std::optional<minifi::utils::Regex> path_regex;
+  std::optional<minifi::utils::Regex> file_regex;
 };
 
 class DataLakeStorageClient {
diff --git a/extensions/civetweb/processors/ListenHTTP.cpp b/extensions/civetweb/processors/ListenHTTP.cpp
index 3853e5ff2..d88f0082a 100644
--- a/extensions/civetweb/processors/ListenHTTP.cpp
+++ b/extensions/civetweb/processors/ListenHTTP.cpp
@@ -319,7 +319,7 @@ void ListenHTTP::Handler::setHeaderAttributes(const mg_request_info *req_info, c
 
     if (strcmp("filename", header->name) == 0) {
       flow_file->setAttribute("filename", header->value);
-    } else if (std::regex_match(header->name, headers_as_attrs_regex_)) {
+    } else if (utils::regexMatch(header->name, headers_as_attrs_regex_)) {
       flow_file->setAttribute(header->name, header->value);
     }
   }
@@ -373,7 +373,7 @@ bool ListenHTTP::Handler::authRequest(mg_connection *conn, const mg_request_info
   // If this is a two-way TLS connection, authorize the peer against the configured pattern
   bool authorized = true;
   if (req_info->is_ssl && req_info->client_cert != nullptr) {
-    if (!std::regex_match(req_info->client_cert->subject, auth_dn_regex_)) {
+    if (!utils::regexMatch(req_info->client_cert->subject, auth_dn_regex_)) {
       mg_printf(conn, "HTTP/1.1 403 Forbidden\r\n"
                 "Content-Type: text/html\r\n"
                 "Content-Length: 0\r\n\r\n");
diff --git a/extensions/civetweb/processors/ListenHTTP.h b/extensions/civetweb/processors/ListenHTTP.h
index a02f7a442..76c9e2c32 100644
--- a/extensions/civetweb/processors/ListenHTTP.h
+++ b/extensions/civetweb/processors/ListenHTTP.h
@@ -21,7 +21,6 @@
 
 #include <map>
 #include <memory>
-#include <regex>
 #include <string>
 #include <utility>
 
@@ -35,6 +34,7 @@
 #include "utils/MinifiConcurrentQueue.h"
 #include "utils/gsl.h"
 #include "utils/Export.h"
+#include "utils/RegexUtils.h"
 
 namespace org {
 namespace apache {
@@ -116,8 +116,8 @@ class ListenHTTP : public core::Processor {
     void enqueueRequest(mg_connection *conn, const mg_request_info *req_info, std::unique_ptr<io::BufferStream>);
 
     std::string base_uri_;
-    std::regex auth_dn_regex_;
-    std::regex headers_as_attrs_regex_;
+    utils::Regex auth_dn_regex_;
+    utils::Regex headers_as_attrs_regex_;
     core::ProcessContext *process_context_;
     std::shared_ptr<core::logging::Logger> logger_ = core::logging::LoggerFactory<ListenHTTP>::getLogger();
     std::map<std::string, ResponseBody> response_uri_map_;
diff --git a/extensions/expression-language/Expression.cpp b/extensions/expression-language/Expression.cpp
index e4168a765..fbce1b104 100644
--- a/extensions/expression-language/Expression.cpp
+++ b/extensions/expression-language/Expression.cpp
@@ -32,6 +32,7 @@
 #include "utils/StringUtils.h"
 #include "utils/OsUtils.h"
 #include "expression/Expression.h"
+#include "utils/RegexUtils.h"
 
 #ifndef DISABLE_CURL
 #ifdef WIN32
@@ -822,16 +823,16 @@ Value expr_replaceEmpty(const std::vector<Value> &args) {
 
 Value expr_matches(const std::vector<Value> &args) {
   const auto &subject = args[0].asString();
-  const std::regex expr = std::regex(args[1].asString());
+  const auto expr = utils::Regex(args[1].asString());
 
-  return Value(std::regex_match(subject.begin(), subject.end(), expr));
+  return Value(utils::regexMatch(subject, expr));
 }
 
 Value expr_find(const std::vector<Value> &args) {
   const auto &subject = args[0].asString();
-  const std::regex expr = std::regex(args[1].asString());
+  const auto expr = utils::Regex(args[1].asString());
 
-  return Value(std::regex_search(subject.begin(), subject.end(), expr));
+  return Value(utils::regexSearch(subject, expr));
 }
 
 #endif  // EXPRESSION_LANGUAGE_USE_REGEX
@@ -1184,7 +1185,7 @@ Expression make_allMatchingAttributes(const std::string &function_name, const st
     std::vector<Expression> out_exprs;
 
     for (const auto &arg : args) {
-      const std::regex attr_regex = std::regex(arg(params).asString());
+      const auto attr_regex = utils::Regex(arg(params).asString());
       const auto cur_flow_file = params.flow_file.lock();
       std::map<std::string, std::string> attrs;
 
@@ -1193,7 +1194,7 @@ Expression make_allMatchingAttributes(const std::string &function_name, const st
       }
 
       for (const auto &attr : attrs) {
-        if (std::regex_match(attr.first.begin(), attr.first.end(), attr_regex)) {
+        if (utils::regexMatch(attr.first, attr_regex)) {
           out_exprs.emplace_back(make_dynamic([=](const Parameters& /*params*/,
                       const std::vector<Expression>& /*sub_exprs*/) -> Value {
                     std::string attr_val;
@@ -1237,7 +1238,7 @@ Expression make_anyMatchingAttribute(const std::string &function_name, const std
     std::vector<Expression> out_exprs;
 
     for (const auto &arg : args) {
-      const std::regex attr_regex = std::regex(arg(params).asString());
+      const auto attr_regex = utils::Regex(arg(params).asString());
       const auto cur_flow_file = params.flow_file.lock();
       std::map<std::string, std::string> attrs;
 
@@ -1246,7 +1247,7 @@ Expression make_anyMatchingAttribute(const std::string &function_name, const std
       }
 
       for (const auto &attr : attrs) {
-        if (std::regex_match(attr.first.begin(), attr.first.end(), attr_regex)) {
+        if (utils::regexMatch(attr.first, attr_regex)) {
           out_exprs.emplace_back(make_dynamic([=](const Parameters& /*params*/,
                       const std::vector<Expression>& /*sub_exprs*/) -> Value {
                     std::string attr_val;
diff --git a/extensions/http-curl/client/HTTPClient.cpp b/extensions/http-curl/client/HTTPClient.cpp
index a63c6bb66..16298aa24 100644
--- a/extensions/http-curl/client/HTTPClient.cpp
+++ b/extensions/http-curl/client/HTTPClient.cpp
@@ -23,12 +23,12 @@
 #include <vector>
 #include <string>
 #include <algorithm>
-#include <regex>
 
 #include "Exception.h"
 #include "utils/gsl.h"
 #include "utils/StringUtils.h"
 #include "core/Resource.h"
+#include "utils/RegexUtils.h"
 
 namespace org {
 namespace apache {
@@ -394,9 +394,9 @@ bool HTTPClient::matches(const std::string &value, const std::string &sregex) {
   if (sregex == ".*")
     return true;
   try {
-    std::regex rgx(sregex);
-    return std::regex_search(value, rgx);
-  } catch (const std::regex_error &) {
+    utils::Regex rgx(sregex);
+    return utils::regexSearch(value, rgx);
+  } catch (const Exception &) {
     return false;
   }
 }
diff --git a/extensions/kubernetes/controllerservice/KubernetesControllerService.cpp b/extensions/kubernetes/controllerservice/KubernetesControllerService.cpp
index 3322db063..18d8fbb8e 100644
--- a/extensions/kubernetes/controllerservice/KubernetesControllerService.cpp
+++ b/extensions/kubernetes/controllerservice/KubernetesControllerService.cpp
@@ -116,17 +116,17 @@ void KubernetesControllerService::onEnable() {
 
   std::string namespace_filter;
   if (getProperty(NamespaceFilter.getName(), namespace_filter) && !namespace_filter.empty()) {
-    namespace_filter_ = std::regex{namespace_filter};
+    namespace_filter_ = utils::Regex{namespace_filter};
   }
 
   std::string pod_name_filter;
   if (getProperty(PodNameFilter.getName(), pod_name_filter) && !pod_name_filter.empty()) {
-    pod_name_filter_ = std::regex{pod_name_filter};
+    pod_name_filter_ = utils::Regex{pod_name_filter};
   }
 
   std::string container_name_filter;
   if (getProperty(ContainerNameFilter.getName(), container_name_filter) && !container_name_filter.empty()) {
-    container_name_filter_ = std::regex{container_name_filter};
+    container_name_filter_ = utils::Regex{container_name_filter};
   }
 }
 
@@ -199,8 +199,8 @@ std::optional<std::vector<KubernetesControllerService::AttributeMap>> Kubernetes
 }
 
 bool KubernetesControllerService::matchesRegexFilters(const std::string& name_space, const std::string& pod_name, const std::string& container_name) const {
-  static constexpr auto matchesFilter = [](const std::string& target, const std::optional<std::regex>& filter) {
-    return !filter || std::regex_match(target, *filter);
+  static constexpr auto matchesFilter = [](const std::string& target, const std::optional<utils::Regex>& filter) {
+    return !filter || utils::regexMatch(target, *filter);
   };
   return matchesFilter(name_space, namespace_filter_) &&
       matchesFilter(pod_name, pod_name_filter_) &&
diff --git a/extensions/kubernetes/controllerservice/KubernetesControllerService.h b/extensions/kubernetes/controllerservice/KubernetesControllerService.h
index e923749c1..51bbc8777 100644
--- a/extensions/kubernetes/controllerservice/KubernetesControllerService.h
+++ b/extensions/kubernetes/controllerservice/KubernetesControllerService.h
@@ -24,6 +24,7 @@
 #include "controllers/AttributeProviderService.h"
 #include "core/logging/Logger.h"
 #include "core/Property.h"
+#include "utils/RegexUtils.h"
 
 namespace org::apache::nifi::minifi::controllers {
 
@@ -48,9 +49,9 @@ class KubernetesControllerService : public AttributeProviderService {
 
   std::mutex initialization_mutex_;
   bool initialized_ = false;
-  std::optional<std::regex> namespace_filter_;
-  std::optional<std::regex> pod_name_filter_;
-  std::optional<std::regex> container_name_filter_;
+  std::optional<utils::Regex> namespace_filter_;
+  std::optional<utils::Regex> pod_name_filter_;
+  std::optional<utils::Regex> container_name_filter_;
   std::shared_ptr<core::logging::Logger> logger_;
   std::unique_ptr<APIClient> api_client_;
 };
diff --git a/extensions/librdkafka/PublishKafka.cpp b/extensions/librdkafka/PublishKafka.cpp
index a45eba348..39111795b 100644
--- a/extensions/librdkafka/PublishKafka.cpp
+++ b/extensions/librdkafka/PublishKafka.cpp
@@ -283,12 +283,12 @@ class ReadCallback : public InputStreamCallback {
     });
   }
 
-  static rd_kafka_headers_unique_ptr make_headers(const core::FlowFile& flow_file, std::regex& attribute_name_regex) {
+  static rd_kafka_headers_unique_ptr make_headers(const core::FlowFile& flow_file, utils::Regex& attribute_name_regex) {
     const gsl::owner<rd_kafka_headers_t*> result{ rd_kafka_headers_new(8) };
     if (!result) { throw std::bad_alloc{}; }
 
     for (const auto& kv : flow_file.getAttributes()) {
-      if (std::regex_search(kv.first, attribute_name_regex)) {
+      if (utils::regexSearch(kv.first, attribute_name_regex)) {
         rd_kafka_header_add(result, kv.first.c_str(), kv.first.size(), kv.second.c_str(), kv.second.size());
       }
     }
@@ -337,7 +337,7 @@ class ReadCallback : public InputStreamCallback {
       rd_kafka_topic_t* const rkt,
       rd_kafka_t* const rk,
       const core::FlowFile& flowFile,
-      std::regex& attributeNameRegex,
+      utils::Regex& attributeNameRegex,
       std::shared_ptr<PublishKafka::Messages> messages,
       const size_t flow_file_index,
       const bool fail_empty_flow_files,
@@ -510,7 +510,7 @@ void PublishKafka::onSchedule(const std::shared_ptr<core::ProcessContext> &conte
   // Attributes to Send as Headers
   std::string value;
   if (context->getProperty(AttributeNameRegex.getName(), value) && !value.empty()) {
-    attributeNameRegex_ = std::regex(value);
+    attributeNameRegex_ = utils::Regex(value);
     logger_->log_debug("PublishKafka: AttributeNameRegex [%s]", value);
   }
 
diff --git a/extensions/librdkafka/PublishKafka.h b/extensions/librdkafka/PublishKafka.h
index 3108c548a..153cc1955 100644
--- a/extensions/librdkafka/PublishKafka.h
+++ b/extensions/librdkafka/PublishKafka.h
@@ -30,7 +30,6 @@
 #include <condition_variable>
 #include <utility>
 #include <vector>
-#include <regex>
 
 #include "KafkaProcessorBase.h"
 #include "utils/GeneralUtils.h"
@@ -43,6 +42,7 @@
 #include "controllers/SSLContextService.h"
 #include "rdkafka.h"
 #include "KafkaConnection.h"
+#include "utils/RegexUtils.h"
 
 namespace org {
 namespace apache {
@@ -133,7 +133,7 @@ class PublishKafka : public KafkaProcessorBase {
   uint32_t batch_size_{};
   uint64_t target_batch_payload_size_{};
   uint64_t max_flow_seg_size_{};
-  std::regex attributeNameRegex_;
+  utils::Regex attributeNameRegex_;
 
   std::atomic<bool> interrupted_{false};
   std::mutex messages_mutex_;  // If both connection_mutex_ and messages_mutex_ are needed, always take connection_mutex_ first to avoid deadlock
diff --git a/extensions/pcap/CapturePacket.cpp b/extensions/pcap/CapturePacket.cpp
index 2a5ae0213..96c1eaa0d 100644
--- a/extensions/pcap/CapturePacket.cpp
+++ b/extensions/pcap/CapturePacket.cpp
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-#include <regex>
 #include <memory>
 #include <algorithm>
 #include <cctype>
@@ -43,6 +42,7 @@
 #include "ResourceClaim.h"
 #include "utils/StringUtils.h"
 #include "utils/ByteArrayCallback.h"
+#include "utils/RegexUtils.h"
 
 namespace org {
 namespace apache {
@@ -167,9 +167,9 @@ void CapturePacket::onSchedule(const std::shared_ptr<core::ProcessContext> &cont
       bool found_match = false;
       std::string matching_regex = "";
       for (const auto &filter : allowed_interfaces) {
-        std::regex r(filter);
-        std::smatch m;
-        if (std::regex_match(name, m, r)) {
+        utils::Regex r(filter);
+        utils::SMatch m;
+        if (utils::regexMatch(name, m, r)) {
           matching_regex = filter;
           found_match = true;
           break;
diff --git a/extensions/sftp/processors/ListSFTP.cpp b/extensions/sftp/processors/ListSFTP.cpp
index 01bb1663b..28a256ae2 100644
--- a/extensions/sftp/processors/ListSFTP.cpp
+++ b/extensions/sftp/processors/ListSFTP.cpp
@@ -34,7 +34,6 @@
 #include <vector>
 #include <tuple>
 #include <deque>
-#include <regex>
 
 #include "utils/ByteArrayCallback.h"
 #include "utils/TimeUtil.h"
@@ -236,9 +235,9 @@ void ListSFTP::onSchedule(const std::shared_ptr<core::ProcessContext> &context,
   }
   if (context->getProperty(FileFilterRegex.getName(), file_filter_regex_)) {
     try {
-      compiled_file_filter_regex_ = std::regex(file_filter_regex_);
+      compiled_file_filter_regex_ = utils::Regex(file_filter_regex_);
       file_filter_regex_set_ = true;
-    } catch (const std::regex_error &e) {
+    } catch (const Exception &e) {
       logger_->log_error("Failed to compile File Filter Regex \"%s\"", file_filter_regex_.c_str());
       file_filter_regex_set_ = false;
     }
@@ -247,9 +246,9 @@ void ListSFTP::onSchedule(const std::shared_ptr<core::ProcessContext> &context,
   }
   if (context->getProperty(PathFilterRegex.getName(), path_filter_regex_)) {
     try {
-      compiled_path_filter_regex_ = std::regex(path_filter_regex_);
+      compiled_path_filter_regex_ = utils::Regex(path_filter_regex_);
       path_filter_regex_set_ = true;
-    } catch (const std::regex_error &e) {
+    } catch (const Exception &e) {
       logger_->log_error("Failed to compile Path Filter Regex \"%s\"", path_filter_regex_.c_str());
       path_filter_regex_set_ = false;
     }
@@ -398,7 +397,7 @@ bool ListSFTP::filterFile(const std::string& parent_path, const std::string& fil
   /* File Filter Regex */
   if (file_filter_regex_set_) {
     bool match = false;
-    match = std::regex_search(filename, compiled_file_filter_regex_);
+    match = utils::regexSearch(filename, compiled_file_filter_regex_);
     if (!match) {
       logger_->log_debug("Ignoring \"%s/%s\" because it did not match the File Filter Regex \"%s\"",
                          parent_path.c_str(),
@@ -420,7 +419,7 @@ bool ListSFTP::filterDirectory(const std::string& parent_path, const std::string
   if (path_filter_regex_set_) {
     std::string dir_path = utils::file::FileUtils::concat_path(parent_path, filename, true /*force_posix*/);
     bool match = false;
-    match = std::regex_search(dir_path, compiled_path_filter_regex_);
+    match = utils::regexSearch(dir_path, compiled_path_filter_regex_);
     if (!match) {
       logger_->log_debug("Not recursing into \"%s\" because it did not match the Path Filter Regex \"%s\"",
                          dir_path.c_str(),
diff --git a/extensions/sftp/processors/ListSFTP.h b/extensions/sftp/processors/ListSFTP.h
index d2f7ac52e..a416a270d 100644
--- a/extensions/sftp/processors/ListSFTP.h
+++ b/extensions/sftp/processors/ListSFTP.h
@@ -25,7 +25,6 @@
 #include <set>
 #include <tuple>
 #include <vector>
-#include <regex>
 
 #include "SFTPProcessorBase.h"
 #include "core/Processor.h"
@@ -33,6 +32,7 @@
 #include "core/Property.h"
 #include "utils/Id.h"
 #include "controllers/keyvalue/PersistableKeyValueStoreService.h"
+#include "utils/RegexUtils.h"
 
 namespace org {
 namespace apache {
@@ -114,8 +114,8 @@ class ListSFTP : public SFTPProcessorBase {
   std::string path_filter_regex_;
   bool file_filter_regex_set_;
   bool path_filter_regex_set_;
-  std::regex compiled_file_filter_regex_;
-  std::regex compiled_path_filter_regex_;
+  utils::Regex compiled_file_filter_regex_;
+  utils::Regex compiled_path_filter_regex_;
   bool ignore_dotted_files_;
   std::string target_system_timestamp_precision_;
   std::string entity_tracking_initial_listing_target_;
diff --git a/extensions/standard-processors/processors/AttributesToJSON.cpp b/extensions/standard-processors/processors/AttributesToJSON.cpp
index 7f8b350be..69dd573d6 100644
--- a/extensions/standard-processors/processors/AttributesToJSON.cpp
+++ b/extensions/standard-processors/processors/AttributesToJSON.cpp
@@ -89,7 +89,7 @@ void AttributesToJSON::onSchedule(core::ProcessContext* context, core::ProcessSe
     attribute_list_ = utils::StringUtils::splitAndTrimRemovingEmpty(value, ",");
   }
   if (context->getProperty(AttributesRegularExpression.getName(), value) && !value.empty()) {
-    attributes_regular_expression_ = std::regex(value);
+    attributes_regular_expression_ = utils::Regex(value);
   }
   write_destination_ = WriteDestination::parse(utils::parsePropertyWithAllowableValuesOrThrow(*context, Destination.getName(), WriteDestination::values()).c_str());
   context->getProperty(IncludeCoreAttributes.getName(), include_core_attributes_);
@@ -114,7 +114,7 @@ std::optional<std::unordered_set<std::string>> AttributesToJSON::getAttributesTo
 
   if (attributes_regular_expression_) {
     for (const auto& [key, value] : flowfile_attributes) {
-      if (std::regex_match(key, attributes_regular_expression_.value())) {
+      if (utils::regexMatch(key, attributes_regular_expression_.value())) {
         attributes.insert(key);
       }
     }
diff --git a/extensions/standard-processors/processors/AttributesToJSON.h b/extensions/standard-processors/processors/AttributesToJSON.h
index d04cd2ce5..35149adad 100644
--- a/extensions/standard-processors/processors/AttributesToJSON.h
+++ b/extensions/standard-processors/processors/AttributesToJSON.h
@@ -25,7 +25,6 @@
 #include <unordered_set>
 #include <memory>
 #include <map>
-#include <regex>
 
 #include "rapidjson/document.h"
 #include "core/FlowFile.h"
@@ -35,6 +34,7 @@
 #include "io/StreamPipe.h"
 #include "utils/Enum.h"
 #include "utils/Export.h"
+#include "utils/RegexUtils.h"
 
 namespace org {
 namespace apache {
@@ -90,7 +90,7 @@ class AttributesToJSON : public core::Processor {
 
   std::shared_ptr<core::logging::Logger> logger_ = core::logging::LoggerFactory<AttributesToJSON>::getLogger();
   std::vector<std::string> attribute_list_;
-  std::optional<std::regex> attributes_regular_expression_;
+  std::optional<utils::Regex> attributes_regular_expression_;
   WriteDestination write_destination_;
   bool include_core_attributes_ = true;
   bool null_value_ = false;
diff --git a/extensions/standard-processors/processors/DefragmentText.cpp b/extensions/standard-processors/processors/DefragmentText.cpp
index fda9dc071..3d5411e0a 100644
--- a/extensions/standard-processors/processors/DefragmentText.cpp
+++ b/extensions/standard-processors/processors/DefragmentText.cpp
@@ -81,7 +81,7 @@ void DefragmentText::onSchedule(core::ProcessContext* context, core::ProcessSess
 
   std::string pattern_str;
   if (context->getProperty(Pattern.getName(), pattern_str) && !pattern_str.empty()) {
-    pattern_ = std::regex(pattern_str);
+    pattern_ = utils::Regex(pattern_str);
     logger_->log_trace("The Pattern is configured to be %s", pattern_str);
   } else {
     throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Pattern property missing or invalid");
@@ -204,7 +204,7 @@ struct ReadFlowFileContent : public InputStreamCallback {
   }
 };
 
-size_t getSplitPosition(const std::smatch& last_match, DefragmentText::PatternLocation pattern_location) {
+size_t getSplitPosition(const utils::SMatch& last_match, DefragmentText::PatternLocation pattern_location) {
   size_t split_position = last_match.position(0);
   if (pattern_location == DefragmentText::PatternLocation::END_OF_MESSAGE) {
     split_position += last_match.length(0);
@@ -220,7 +220,7 @@ bool DefragmentText::splitFlowFileAtLastPattern(core::ProcessSession *session,
                                                 std::shared_ptr<core::FlowFile> &split_after_last_pattern) const {
   ReadFlowFileContent read_flow_file_content;
   session->read(original_flow_file, &read_flow_file_content);
-  auto last_regex_match = utils::StringUtils::getLastRegexMatch(read_flow_file_content.content, pattern_);
+  auto last_regex_match = utils::getLastRegexMatch(read_flow_file_content.content, pattern_);
   if (!last_regex_match.ready()) {
     split_before_last_pattern = session->clone(original_flow_file);
     split_after_last_pattern = nullptr;
diff --git a/extensions/standard-processors/processors/DefragmentText.h b/extensions/standard-processors/processors/DefragmentText.h
index eef72727f..163510a8b 100644
--- a/extensions/standard-processors/processors/DefragmentText.h
+++ b/extensions/standard-processors/processors/DefragmentText.h
@@ -17,7 +17,6 @@
 
 #pragma once
 
-#include <regex>
 #include <memory>
 #include <string>
 #include <set>
@@ -29,6 +28,7 @@
 #include "core/logging/LoggerConfiguration.h"
 #include "utils/Enum.h"
 #include "serialization/PayloadSerializer.h"
+#include "utils/RegexUtils.h"
 
 namespace org::apache::nifi::minifi::processors {
 
@@ -97,7 +97,7 @@ class DefragmentText : public core::Processor {
   };
 
 
-  std::regex pattern_;
+  utils::Regex pattern_;
   PatternLocation pattern_location_;
   std::optional<std::chrono::milliseconds> max_age_;
   std::optional<size_t> max_size_;
diff --git a/extensions/standard-processors/processors/ExtractText.cpp b/extensions/standard-processors/processors/ExtractText.cpp
index 8763ca40a..68cd6d4e3 100644
--- a/extensions/standard-processors/processors/ExtractText.cpp
+++ b/extensions/standard-processors/processors/ExtractText.cpp
@@ -23,7 +23,6 @@
 #include <memory>
 #include <map>
 #include <set>
-#include <regex>
 #include <iostream>
 #include <sstream>
 #include <utility>
@@ -34,6 +33,7 @@
 #include "core/Resource.h"
 #include "core/FlowFile.h"
 #include "utils/gsl.h"
+#include "utils/RegexUtils.h"
 
 namespace org {
 namespace apache {
@@ -149,9 +149,9 @@ int64_t ExtractText::ReadCallback::process(const std::shared_ptr<io::BaseStream>
 
   if (regex_mode) {
     bool insensitive;
-    std::regex_constants::syntax_option_type regex_flags = std::regex::ECMAScript;  // ECMAScript is the default behaviour
+    std::vector<utils::Regex::Mode> regex_flags;
     if (ctx_->getProperty(InsensitiveMatch.getName(), insensitive) && insensitive) {
-      regex_flags |= std::regex_constants::icase;
+      regex_flags.push_back(utils::Regex::Mode::ICASE);
     }
 
     bool ignoregroupzero;
@@ -179,9 +179,9 @@ int64_t ExtractText::ReadCallback::process(const std::shared_ptr<io::BaseStream>
       int matchcount = 0;
 
       try {
-        std::regex rgx(value, regex_flags);
-        std::smatch matches;
-        while (std::regex_search(workStr, matches, rgx)) {
+        utils::Regex rgx(value, regex_flags);
+        utils::SMatch matches;
+        while (utils::regexSearch(workStr, matches, rgx)) {
           size_t i = ignoregroupzero ? 1 : 0;
 
           for (; i < matches.size(); ++i, ++matchcount) {
@@ -199,7 +199,7 @@ int64_t ExtractText::ReadCallback::process(const std::shared_ptr<io::BaseStream>
           }
           workStr = matches.suffix();
         }
-      } catch (const std::regex_error &e) {
+      } catch (const Exception &e) {
         logger_->log_error("%s error encountered when trying to construct regular expression from property (key: %s) value: %s",
                            e.what(), k, value);
         continue;
diff --git a/extensions/standard-processors/processors/GetFile.cpp b/extensions/standard-processors/processors/GetFile.cpp
index e99afd105..721860f36 100644
--- a/extensions/standard-processors/processors/GetFile.cpp
+++ b/extensions/standard-processors/processors/GetFile.cpp
@@ -27,7 +27,6 @@
 #include <memory>
 #include <set>
 #include <string>
-#include <regex>
 
 #include "utils/StringUtils.h"
 #include "utils/file/FileUtils.h"
@@ -37,6 +36,7 @@
 #include "core/Resource.h"
 #include "core/TypedValues.h"
 #include "utils/FileReaderCallback.h"
+#include "utils/RegexUtils.h"
 
 using namespace std::literals::chrono_literals;
 
@@ -261,8 +261,8 @@ bool GetFile::fileMatchesRequestCriteria(std::string fullName, std::string name,
   if (request.ignoreHiddenFile && utils::file::is_hidden(fullName))
     return false;
 
-  std::regex rgx(request.fileFilter);
-  if (!std::regex_search(name, rgx)) {
+  utils::Regex rgx(request.fileFilter);
+  if (!utils::regexSearch(name, rgx)) {
     return false;
   }
 
diff --git a/extensions/standard-processors/processors/ListenSyslog.h b/extensions/standard-processors/processors/ListenSyslog.h
index d8c92423b..e85deaf52 100644
--- a/extensions/standard-processors/processors/ListenSyslog.h
+++ b/extensions/standard-processors/processors/ListenSyslog.h
@@ -21,6 +21,7 @@
 #include <utility>
 #include <string>
 #include <memory>
+#include <regex>
 
 #include "core/Processor.h"
 #include "core/logging/Logger.h"
diff --git a/extensions/standard-processors/processors/RouteText.cpp b/extensions/standard-processors/processors/RouteText.cpp
index e1eab5c67..3c97339f0 100644
--- a/extensions/standard-processors/processors/RouteText.cpp
+++ b/extensions/standard-processors/processors/RouteText.cpp
@@ -135,7 +135,7 @@ void RouteText::onSchedule(core::ProcessContext* context, core::ProcessSessionFa
   matching_ = utils::parseEnumProperty<Matching>(*context, MatchingStrategy);
   context->getProperty(TrimWhitespace.getName(), trim_);
   case_policy_ = context->getProperty<bool>(IgnoreCase).value_or(false) ? CasePolicy::IGNORE_CASE : CasePolicy::CASE_SENSITIVE;
-  group_regex_ = context->getProperty(GroupingRegex) | utils::map([] (const auto& str) {return std::regex(str);});
+  group_regex_ = context->getProperty(GroupingRegex) | utils::map([] (const auto& str) {return utils::Regex(str);});
   segmentation_ = utils::parseEnumProperty<Segmentation>(*context, SegmentationStrategy);
   context->getProperty(GroupingFallbackValue.getName(), group_fallback_);
 }
@@ -227,7 +227,7 @@ class RouteText::MatchingContext {
       flow_file_(std::move(flow_file)),
       case_policy_(case_policy) {}
 
-  const std::regex& getRegexProperty(const core::Property& prop) {
+  const utils::Regex& getRegexProperty(const core::Property& prop) {
     auto it = regex_values_.find(prop.getName());
     if (it != regex_values_.end()) {
       return it->second;
@@ -236,11 +236,11 @@ class RouteText::MatchingContext {
     if (!process_context_.getDynamicProperty(prop, value, flow_file_)) {
       throw Exception(PROCESSOR_EXCEPTION, "Missing dynamic property: '" + prop.getName() + "'");
     }
-    std::regex::flag_type flags = std::regex::ECMAScript;
+    std::vector<utils::Regex::Mode> flags;
     if (case_policy_ == CasePolicy::IGNORE_CASE) {
-      flags |= std::regex::icase;
+      flags.push_back(utils::Regex::Mode::ICASE);
     }
-    return (regex_values_[prop.getName()] = std::regex(value, flags));
+    return (regex_values_[prop.getName()] = utils::Regex(value, flags));
   }
 
   const std::string& getStringProperty(const core::Property& prop) {
@@ -275,7 +275,7 @@ class RouteText::MatchingContext {
   CasePolicy case_policy_;
 
   std::map<std::string, std::string> string_values_;
-  std::map<std::string, std::regex> regex_values_;
+  std::map<std::string, utils::Regex> regex_values_;
 
   struct OwningSearcher {
     OwningSearcher(std::string str, CasePolicy case_policy)
@@ -425,12 +425,12 @@ bool RouteText::matchSegment(MatchingContext& context, const Segment& segment, c
       return utils::StringUtils::equals(segment.value_, context.getStringProperty(prop), case_policy_ == CasePolicy::CASE_SENSITIVE);
     }
     case Matching::CONTAINS_REGEX: {
-      std::match_results<std::string_view::const_iterator> match_result;
-      return std::regex_search(segment.value_.begin(), segment.value_.end(), match_result, context.getRegexProperty(prop));
+      std::string segment_str = std::string(segment.value_);
+      return utils::regexSearch(segment_str, context.getRegexProperty(prop));
     }
     case Matching::MATCHES_REGEX: {
-      std::match_results<std::string_view::const_iterator> match_result;
-      return std::regex_match(segment.value_.begin(), segment.value_.end(), match_result, context.getRegexProperty(prop));
+      std::string segment_str = std::string(segment.value_);
+      return utils::regexMatch(segment_str, context.getRegexProperty(prop));
     }
   }
   throw Exception(PROCESSOR_EXCEPTION, "Unknown matching strategy");
@@ -440,8 +440,9 @@ std::optional<std::string> RouteText::getGroup(const std::string_view& segment)
   if (!group_regex_) {
     return std::nullopt;
   }
-  std::match_results<std::string_view::const_iterator> match_result;
-  if (!std::regex_match(segment.begin(), segment.end(), match_result, group_regex_.value())) {
+  utils::SMatch match_result;
+  std::string segment_str = std::string(segment);
+  if (!utils::regexMatch(segment_str, match_result, group_regex_.value())) {
     return group_fallback_;
   }
   // WARNING!! using a temporary std::string causes the omission of delimiters
diff --git a/extensions/standard-processors/processors/RouteText.h b/extensions/standard-processors/processors/RouteText.h
index b30dbadf1..636e5b04c 100644
--- a/extensions/standard-processors/processors/RouteText.h
+++ b/extensions/standard-processors/processors/RouteText.h
@@ -17,7 +17,6 @@
 
 #pragma once
 
-#include <regex>
 #include <optional>
 #include <string_view>
 #include <map>
@@ -27,6 +26,7 @@
 #include "Processor.h"
 #include "utils/Enum.h"
 #include "utils/Export.h"
+#include "utils/RegexUtils.h"
 
 namespace org::apache::nifi::minifi::processors {
 
@@ -111,7 +111,7 @@ class RouteText : public core::Processor {
   Segmentation segmentation_;
   bool trim_{true};
   CasePolicy case_policy_{CasePolicy::CASE_SENSITIVE};
-  std::optional<std::regex> group_regex_;
+  std::optional<utils::Regex> group_regex_;
   std::string group_fallback_;
 
   std::map<std::string, core::Property> dynamic_properties_;
diff --git a/extensions/standard-processors/processors/TailFile.cpp b/extensions/standard-processors/processors/TailFile.cpp
index 098e794ce..2e065a6ba 100644
--- a/extensions/standard-processors/processors/TailFile.cpp
+++ b/extensions/standard-processors/processors/TailFile.cpp
@@ -30,7 +30,6 @@
 #include <string>
 #include <utility>
 #include <vector>
-#include <regex>
 
 #include "range/v3/action/sort.hpp"
 #include "range/v3/range/conversion.hpp"
@@ -48,6 +47,7 @@
 #include "core/ProcessContext.h"
 #include "core/ProcessSession.h"
 #include "core/Resource.h"
+#include "utils/RegexUtils.h"
 
 
 namespace org {
@@ -631,8 +631,8 @@ std::vector<TailState> TailFile::findAllRotatedFiles(const TailState &state) con
 
   std::vector<TailStateWithMtime> matched_files_with_mtime;
   auto collect_matching_files = [&](const std::string &path, const std::string &file_name) -> bool {
-    std::regex pattern_regex(pattern);
-    if (file_name != state.file_name_ && std::regex_match(file_name, pattern_regex)) {
+    utils::Regex pattern_regex(pattern);
+    if (file_name != state.file_name_ && utils::regexMatch(file_name, pattern_regex)) {
       std::string full_file_name = path + utils::file::get_separator() + file_name;
       TailStateWithMtime::TimePoint mtime{utils::file::last_write_time_point(full_file_name)};
       logger_->log_debug("File %s with mtime %" PRId64 " matches rolling filename pattern %s, so we are reading it", file_name, int64_t{mtime.time_since_epoch().count()}, pattern);
@@ -653,8 +653,8 @@ std::vector<TailState> TailFile::findRotatedFilesAfterLastReadTime(const TailSta
 
   std::vector<TailStateWithMtime> matched_files_with_mtime;
   auto collect_matching_files = [&](const std::string &path, const std::string &file_name) -> bool {
-    std::regex pattern_regex(pattern);
-    if (file_name != state.file_name_ && std::regex_match(file_name, pattern_regex)) {
+    utils::Regex pattern_regex(pattern);
+    if (file_name != state.file_name_ && utils::regexMatch(file_name, pattern_regex)) {
       std::string full_file_name = path + utils::file::get_separator() + file_name;
       TailStateWithMtime::TimePoint mtime{utils::file::last_write_time_point(full_file_name)};
       logger_->log_debug("File %s with mtime %" PRId64 " matches rolling filename pattern %s", file_name, int64_t{mtime.time_since_epoch().count()}, pattern);
@@ -867,9 +867,9 @@ void TailFile::checkForRemovedFiles() {
   for (const auto &kv : tail_states_) {
     const std::string &full_file_name = kv.first;
     const TailState &state = kv.second;
-    std::regex pattern_regex(file_to_tail_);
+    utils::Regex pattern_regex(file_to_tail_);
     if (utils::file::file_size(state.fileNameWithPath()) == 0u ||
-        !std::regex_match(state.file_name_, pattern_regex)) {
+        !utils::regexMatch(state.file_name_, pattern_regex)) {
       file_names_to_remove.push_back(full_file_name);
     }
   }
@@ -882,8 +882,8 @@ void TailFile::checkForRemovedFiles() {
 void TailFile::checkForNewFiles(core::ProcessContext& context) {
   auto add_new_files_callback = [&](const std::string &path, const std::string &file_name) -> bool {
     std::string full_file_name = path + utils::file::get_separator() + file_name;
-    std::regex file_to_tail_regex(file_to_tail_);
-    if (!containsKey(tail_states_, full_file_name) && std::regex_match(file_name, file_to_tail_regex)) {
+    utils::Regex file_to_tail_regex(file_to_tail_);
+    if (!containsKey(tail_states_, full_file_name) && utils::regexMatch(file_name, file_to_tail_regex)) {
       tail_states_.emplace(full_file_name, TailState{path, file_name});
     }
     return true;
diff --git a/extensions/standard-processors/tests/unit/ExtractTextTests.cpp b/extensions/standard-processors/tests/unit/ExtractTextTests.cpp
index 09a47ca5d..f41f74e7a 100644
--- a/extensions/standard-processors/tests/unit/ExtractTextTests.cpp
+++ b/extensions/standard-processors/tests/unit/ExtractTextTests.cpp
@@ -29,6 +29,7 @@
 #include "core/Core.h"
 #include "unit/ProvenanceTestHelper.h"
 #include "repository/VolatileContentRepository.h"
+#include "utils/TestUtils.h"
 
 #include "core/FlowFile.h"
 #include "core/Processor.h"
@@ -180,3 +181,34 @@ TEST_CASE("Test usage of ExtractText in regex mode", "[extracttextRegexTest]") {
 
   LogTestController::getInstance().reset();
 }
+
+TEST_CASE("Test usage of ExtractText in regex mode with large regex matches", "[extracttextRegexTest]") {
+  TestController test_controller;
+  LogTestController::getInstance().setTrace<org::apache::nifi::minifi::processors::ExtractText>();
+  LogTestController::getInstance().setTrace<org::apache::nifi::minifi::processors::GetFile>();
+  LogTestController::getInstance().setTrace<org::apache::nifi::minifi::processors::LogAttribute>();
+
+  std::shared_ptr<TestPlan> plan = test_controller.createPlan();
+  std::shared_ptr<TestRepository> repo = std::make_shared<TestRepository>();
+
+  auto dir = test_controller.createTempDirectory();
+  REQUIRE(!dir.empty());
+  auto getfile = plan->addProcessor("GetFile", "GetFile");
+  plan->setProperty(getfile, org::apache::nifi::minifi::processors::GetFile::Directory.getName(), dir);
+  plan->setProperty(getfile, org::apache::nifi::minifi::processors::GetFile::KeepSourceFile.getName(), "true");
+
+  auto extract_text_processor = plan->addProcessor("ExtractText", "ExtractText", core::Relationship("success", "description"), true);
+  plan->setProperty(extract_text_processor, org::apache::nifi::minifi::processors::ExtractText::RegexMode.getName(), "true");
+  plan->setProperty(extract_text_processor, "RegexAttr", "Speed limit (.*)", true);
+
+  auto log_attribute_processor = plan->addProcessor("LogAttribute", "outputLogAttribute", core::Relationship("success", "description"), true);
+  plan->setProperty(log_attribute_processor, org::apache::nifi::minifi::processors::LogAttribute::AttributesToLog.getName(), TEST_ATTR);
+
+  std::string additional_long_string(100'000, '.');
+  utils::putFileToDir(dir, TEST_FILE, "Speed limit 80" + additional_long_string);
+
+  test_controller.runSession(plan);
+
+  REQUIRE(LogTestController::getInstance().contains("key:RegexAttr.0 value:80"));
+  LogTestController::getInstance().reset();
+}
diff --git a/extensions/standard-processors/tests/unit/RouteTextTests.cpp b/extensions/standard-processors/tests/unit/RouteTextTests.cpp
index 4e4d1b3a7..b59159cad 100644
--- a/extensions/standard-processors/tests/unit/RouteTextTests.cpp
+++ b/extensions/standard-processors/tests/unit/RouteTextTests.cpp
@@ -402,7 +402,7 @@ TEST_CASE_METHOD(RouteTextController, "RouteText grouping uses empty strings for
   proc_->setProperty(processors::RouteText::RoutingStrategy, "Dynamic Routing");
   proc_->setProperty(processors::RouteText::SegmentationStrategy, "Per Line");
   proc_->setProperty(processors::RouteText::MatchingStrategy, "Contains");
-  proc_->setProperty(processors::RouteText::GroupingRegex, "group(.)(?:\\.(.))?.*");
+  proc_->setProperty(processors::RouteText::GroupingRegex, "group(.)(\\..)?.*");
 
   proc_->setDynamicProperty("A", "toA");
 
@@ -416,8 +416,8 @@ TEST_CASE_METHOD(RouteTextController, "RouteText grouping uses empty strings for
   std::map<std::string, FlowFilePatternVec> expected;
 
   expected["A"] = {
-      FlowFilePattern{}.attr("RouteText.Group", "1, 1").content("group1.1:toA(one)\ngroup1.1:toA(two)\n"),
-      FlowFilePattern{}.attr("RouteText.Group", "1, 2").content("group1.2:toA(three)\n"),
+      FlowFilePattern{}.attr("RouteText.Group", "1, .1").content("group1.1:toA(one)\ngroup1.1:toA(two)\n"),
+      FlowFilePattern{}.attr("RouteText.Group", "1, .2").content("group1.2:toA(three)\n"),
       FlowFilePattern{}.attr("RouteText.Group", "2, ").content("group2:toA(four)\ngroup2:toA(five)")
   };
   expected["matched"] = FlowFilePatternVec{};
diff --git a/extensions/standard-processors/tests/unit/TailFileTests.cpp b/extensions/standard-processors/tests/unit/TailFileTests.cpp
index 089d8121c..e7f54d889 100644
--- a/extensions/standard-processors/tests/unit/TailFileTests.cpp
+++ b/extensions/standard-processors/tests/unit/TailFileTests.cpp
@@ -1824,11 +1824,11 @@ TEST_CASE("TailFile can use an AttributeProviderService", "[AttributeProviderSer
   CHECK(LogTestController::getInstance().contains("key:test.animal value:dog"));
   CHECK(LogTestController::getInstance().contains("key:test.animal value:dolphin"));
 
-  CHECK_FALSE(LogTestController::getInstance().contains("key:test.fruit value:strawberry"));
-  CHECK_FALSE(LogTestController::getInstance().contains("key:test.uid value:002"));
-  CHECK_FALSE(LogTestController::getInstance().contains("key:test.uid value:003"));
-  CHECK_FALSE(LogTestController::getInstance().contains("key:test.animal value:elephant"));
-  CHECK_FALSE(LogTestController::getInstance().contains("key:test.animal value:horse"));
+  CHECK_FALSE(LogTestController::getInstance().contains("key:test.fruit value:strawberry", 0s, 0ms));
+  CHECK_FALSE(LogTestController::getInstance().contains("key:test.uid value:002", 0s, 0ms));
+  CHECK_FALSE(LogTestController::getInstance().contains("key:test.uid value:003", 0s, 0ms));
+  CHECK_FALSE(LogTestController::getInstance().contains("key:test.animal value:elephant", 0s, 0ms));
+  CHECK_FALSE(LogTestController::getInstance().contains("key:test.animal value:horse", 0s, 0ms));
 
   LogTestController::getInstance().reset();
 }
diff --git a/extensions/windows-event-log/wel/MetadataWalker.cpp b/extensions/windows-event-log/wel/MetadataWalker.cpp
index 7d68676bf..9ac4f02ed 100644
--- a/extensions/windows-event-log/wel/MetadataWalker.cpp
+++ b/extensions/windows-event-log/wel/MetadataWalker.cpp
@@ -52,11 +52,11 @@ bool MetadataWalker::for_each(pugi::xml_node &node) {
         return input;
       };
 
-      if (std::regex_match(attr.name(), regex_)) {
+      if (utils::regexMatch(attr.name(), regex_)) {
         updateText(node, attr.name(), idUpdate);
       }
 
-      if (std::regex_match(attr.value(), regex_)) {
+      if (utils::regexMatch(attr.value(), regex_)) {
         updateText(node, attr.value(), idUpdate);
       }
     }
diff --git a/extensions/windows-event-log/wel/MetadataWalker.h b/extensions/windows-event-log/wel/MetadataWalker.h
index f779a7d2d..f4938d7a9 100644
--- a/extensions/windows-event-log/wel/MetadataWalker.h
+++ b/extensions/windows-event-log/wel/MetadataWalker.h
@@ -27,7 +27,6 @@
 #include <map>
 #include <sstream>
 #include <string>
-#include <regex>
 #include <vector>
 
 #include "core/Core.h"
@@ -39,6 +38,7 @@
 
 #include "concurrentqueue.h"
 #include "pugixml.hpp"
+#include "utils/RegexUtils.h"
 
 namespace org {
 namespace apache {
@@ -97,7 +97,7 @@ class MetadataWalker : public pugi::xml_tree_walker {
 
   const WindowsEventLogMetadata& windows_event_log_metadata_;
   std::string log_name_;
-  std::regex regex_;
+  utils::Regex regex_;
   std::string regex_str_;
   bool update_xml_;
   bool resolve_;
diff --git a/libminifi/include/utils/HTTPUtils.h b/libminifi/include/utils/HTTPUtils.h
index 275cda2d4..8e62af8e3 100644
--- a/libminifi/include/utils/HTTPUtils.h
+++ b/libminifi/include/utils/HTTPUtils.h
@@ -20,9 +20,9 @@
 #define LIBMINIFI_INCLUDE_UTILS_HTTPUTILS_H_
 
 #include <string>
-#include <regex>
 
 #include "io/ClientSocket.h"
+#include "utils/RegexUtils.h"
 
 /**
 This function, unfortunately, assumes that we're parsing http components of a local host. On windows this is problematic
@@ -36,9 +36,9 @@ inline bool parse_http_components(const std::string &url, std::string &port, std
   std::string regex_str = "^(http|https)://(localhost:)([0-9]+)?(/.*)$";
 #endif
 
-  auto rgx = std::regex(regex_str, std::regex_constants::icase);
-  std::smatch matches;
-  if (std::regex_search(url, matches, rgx)) {
+  auto rgx = org::apache::nifi::minifi::utils::Regex(regex_str, {org::apache::nifi::minifi::utils::Regex::Mode::ICASE});
+  org::apache::nifi::minifi::utils::SMatch matches;
+  if (org::apache::nifi::minifi::utils::regexSearch(url, matches, rgx)) {
     if (matches.size() >= 5) {
       scheme = matches[1];
       port = matches[3];
diff --git a/libminifi/include/utils/RegexUtils.h b/libminifi/include/utils/RegexUtils.h
new file mode 100644
index 000000000..f0b7cdf0f
--- /dev/null
+++ b/libminifi/include/utils/RegexUtils.h
@@ -0,0 +1,173 @@
+/**
+ *
+ * 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 <string>
+#include <string_view>
+#include <vector>
+#include <cstddef>
+
+// There is a bug in std::regex implementation of libstdc++ which causes stack overflow on long matches: https://gcc.gnu.org/bugzilla/show_bug.cgi?id=86164
+// Due to this bug we should use regex.h for regex searches if libstdc++ is used until a fix is released.
+#if defined(__GLIBCXX__) || defined(__GLIBCPP__)
+#include <regex.h>
+#else
+#include <regex>
+#define NO_MORE_REGFREEE
+#endif
+
+namespace org::apache::nifi::minifi::utils {
+
+class Regex;
+
+#ifdef NO_MORE_REGFREEE
+using SMatch = std::smatch;
+#else
+class SMatch {
+  struct Regmatch;
+  struct SuffixWrapper;
+ public:
+  struct Iterator {
+    using iterator_category = std::forward_iterator_tag;
+    using difference_type   = std::ptrdiff_t;
+    using value_type        = Regmatch;
+    using pointer           = value_type*;
+    using reference         = value_type&;
+
+    Iterator() : regmatch_(nullptr) {
+    }
+
+    explicit Iterator(Regmatch* regmatch)
+      : regmatch_(regmatch) {
+    }
+
+    reference operator*() const { return *regmatch_; }
+    pointer operator->() { return regmatch_; }
+
+    Iterator& operator++() { regmatch_++; return *this; }
+    Iterator operator++(int) { Iterator tmp = *this; ++(*this); return tmp; }
+
+    friend bool operator== (const Iterator& a, const Iterator& b) { return a.regmatch_ == b.regmatch_; }
+    friend bool operator!= (const Iterator& a, const Iterator& b) { return a.regmatch_ != b.regmatch_; }
+
+   private:
+    pointer regmatch_;
+  };
+
+  SuffixWrapper suffix() const;
+  const Regmatch& operator[](std::size_t index) const;
+  Iterator begin() { return Iterator(&matches_[0]); }
+  Iterator end() { return Iterator(&matches_[matches_.size()]); }
+
+  std::size_t size() const;
+  bool ready() const;
+  std::size_t position(std::size_t index) const;
+  std::size_t length(std::size_t index) const;
+
+ private:
+  struct Regmatch {
+    operator std::string() const {
+      return str();
+    }
+
+    std::string str() const {
+      if (match.rm_so == -1) {
+        return "";
+      }
+      return std::string(pattern.begin() + match.rm_so, pattern.begin() + match.rm_eo);
+    }
+
+    regmatch_t match;
+    std::string_view pattern;
+  };
+
+  struct SuffixWrapper {
+    operator std::string() const {
+      return str();
+    }
+
+    std::string str() const {
+      return suffix;
+    }
+
+    std::string suffix;
+  };
+
+  void clear();
+
+  std::vector<Regmatch> matches_;
+  std::string pattern_;
+
+  friend bool regexMatch(const std::string &pattern, SMatch& match, const Regex& regex);
+  friend bool regexSearch(const std::string &pattern, SMatch& match, const Regex& regex);
+  friend utils::SMatch getLastRegexMatch(const std::string& str, const utils::Regex& pattern);
+};
+#endif
+
+class Regex {
+ public:
+  enum class Mode { ICASE };
+
+  Regex();
+  explicit Regex(const std::string &value);
+  explicit Regex(const std::string &value,
+                const std::vector<Mode> &mode);
+  Regex(const Regex &);
+  Regex& operator=(const Regex &);
+  Regex(Regex&& other);
+  Regex& operator=(Regex&& other);
+  ~Regex();
+
+ private:
+  std::string regex_str_;
+  bool valid_;
+
+#ifdef NO_MORE_REGFREEE
+  std::regex compiled_regex_;
+  std::regex_constants::syntax_option_type regex_mode_;
+#else
+  void compileRegex(regex_t& regex, const std::string& regex_string) const;
+
+  regex_t compiled_regex_;
+  regex_t compiled_full_input_regex_;
+  int regex_mode_;
+#endif
+
+  friend bool regexMatch(const std::string &pattern, const Regex& regex);
+  friend bool regexMatch(const std::string &pattern, SMatch& match, const Regex& regex);
+  friend bool regexSearch(const std::string &pattern, const Regex& regex);
+  friend bool regexSearch(const std::string &pattern, SMatch& match, const Regex& regex);
+  friend SMatch getLastRegexMatch(const std::string& pattern, const utils::Regex& regex);
+};
+
+bool regexMatch(const std::string &pattern, const Regex& regex);
+bool regexMatch(const std::string &pattern, SMatch& match, const Regex& regex);
+
+bool regexSearch(const std::string &pattern, const Regex& regex);
+bool regexSearch(const std::string &pattern, SMatch& match, const Regex& regex);
+
+/**
+ * Returns the last match of a regular expression within the given string
+ * @param pattern incoming string
+ * @param regex the regex to be matched
+ * @return the last valid SMatch or a default constructed SMatch (ready() != true) if no matches have been found
+ */
+SMatch getLastRegexMatch(const std::string& pattern, const utils::Regex& regex);
+
+}  // namespace org::apache::nifi::minifi::utils
diff --git a/libminifi/include/utils/StringUtils.h b/libminifi/include/utils/StringUtils.h
index 5b1902e42..d43486bc2 100644
--- a/libminifi/include/utils/StringUtils.h
+++ b/libminifi/include/utils/StringUtils.h
@@ -23,7 +23,6 @@
 #include <iostream>
 #include <map>
 #include <optional>
-#include <regex>
 #include <sstream>
 #include <string>
 #include <string_view>
@@ -485,14 +484,6 @@ class StringUtils {
     return str;
   }
 
-  /**
-   * Returns the last match of a regular expression within the given string
-   * @param str incoming string
-   * @param pattern the regex to be matched
-   * @return the last valid std::smatch or a default constructed smatch (ready() != true) if no matches have been found
-   */
-  static std::smatch getLastRegexMatch(const std::string& str, const std::regex& pattern);
-
   static std::string escapeUnprintableBytes(gsl::span<const std::byte> data);
 
  private:
diff --git a/libminifi/src/core/yaml/YamlConfiguration.cpp b/libminifi/src/core/yaml/YamlConfiguration.cpp
index 4d09d49c1..299c7c8f9 100644
--- a/libminifi/src/core/yaml/YamlConfiguration.cpp
+++ b/libminifi/src/core/yaml/YamlConfiguration.cpp
@@ -29,7 +29,7 @@
 #include "utils/TimeUtil.h"
 
 #ifdef YAML_CONFIGURATION_USE_REGEX
-#include <regex>
+#include "utils/RegexUtils.h"
 #endif  // YAML_CONFIGURATION_USE_REGEX
 
 namespace org {
@@ -826,8 +826,8 @@ void YamlConfiguration::validateComponentProperties(ConfigurableComponent& compo
     }
 
     for (const auto &excl_pair : excl_props) {
-      std::regex excl_expr(excl_pair.second);
-      if (std::regex_match(component_properties.at(excl_pair.first).getValue().to_string(), excl_expr)) {
+      utils::Regex excl_expr(excl_pair.second);
+      if (utils::regexMatch(component_properties.at(excl_pair.first).getValue().to_string(), excl_expr)) {
         std::string reason = utils::StringUtils::join_pack("property '", prop_pair.second.getName(),
             "' must not be set when the value of property '", excl_pair.first, "' matches '", excl_pair.second, "'");
         raiseComponentError(component_name, yaml_section, reason);
@@ -840,8 +840,8 @@ void YamlConfiguration::validateComponentProperties(ConfigurableComponent& compo
     const auto &prop_regex_str = prop_pair.second.getValidRegex();
 
     if (!prop_regex_str.empty()) {
-      std::regex prop_regex(prop_regex_str);
-      if (!std::regex_match(prop_pair.second.getValue().to_string(), prop_regex)) {
+      utils::Regex prop_regex(prop_regex_str);
+      if (!utils::regexMatch(prop_pair.second.getValue().to_string(), prop_regex)) {
         std::string reason = utils::StringUtils::join_pack("property '", prop_pair.second.getName(), "' does not match validation pattern '", prop_regex_str, "'");
         raiseComponentError(component_name, yaml_section, reason);
       }
diff --git a/libminifi/src/utils/RegexUtils.cpp b/libminifi/src/utils/RegexUtils.cpp
new file mode 100644
index 000000000..0b9d9d38f
--- /dev/null
+++ b/libminifi/src/utils/RegexUtils.cpp
@@ -0,0 +1,295 @@
+/**
+ *
+ * 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 "utils/RegexUtils.h"
+
+#include <iostream>
+#include <vector>
+
+#include "Exception.h"
+
+#ifndef NO_MORE_REGFREEE
+namespace {
+
+std::size_t getMaxGroupCountOfRegex(const std::string& regex) {
+  return std::count(regex.begin(), regex.end(), '(') + 1;
+}
+
+}  // namespace
+#endif
+
+namespace org::apache::nifi::minifi::utils {
+
+#ifndef NO_MORE_REGFREEE
+SMatch::SuffixWrapper SMatch::suffix() const {
+  if ((size_t) matches_[0].match.rm_eo >= pattern_.size()) {
+    return SuffixWrapper{std::string()};
+  } else {
+    return SuffixWrapper{pattern_.substr(matches_[0].match.rm_eo)};
+  }
+}
+
+const SMatch::Regmatch& SMatch::operator[](std::size_t index) const {
+  return matches_[index];
+}
+
+std::size_t SMatch::size() const {
+  std::size_t count = 0;
+  for (const auto &m : matches_) {
+    if (m.match.rm_so == -1) {
+      break;
+    }
+    ++count;
+  }
+  return count;
+}
+
+bool SMatch::ready() const {
+  return !matches_.empty();
+}
+
+std::size_t SMatch::position(std::size_t index) const {
+  return matches_.at(index).match.rm_so;
+}
+
+std::size_t SMatch::length(std::size_t index) const {
+  return matches_.at(index).match.rm_eo - matches_.at(index).match.rm_so;
+}
+
+void SMatch::clear() {
+  matches_.clear();
+  pattern_.clear();
+}
+#endif
+
+Regex::Regex() : Regex::Regex("") {}
+
+Regex::Regex(const std::string &value) : Regex::Regex(value, {}) {}
+
+Regex::Regex(const std::string &value,
+             const std::vector<Regex::Mode> &mode)
+    : regex_str_(value),
+      valid_(false) {
+  // Create regex mode
+#ifdef NO_MORE_REGFREEE
+  regex_mode_ = std::regex_constants::ECMAScript;
+#else
+  regex_mode_ = REG_EXTENDED;
+#endif
+  for (const auto m : mode) {
+    switch (m) {
+      case Mode::ICASE:
+#ifdef NO_MORE_REGFREEE
+        regex_mode_ |= std::regex_constants::icase;
+#else
+        regex_mode_ |= REG_ICASE;
+#endif
+        break;
+    }
+  }
+#ifdef NO_MORE_REGFREEE
+  try {
+    compiled_regex_ = std::regex(regex_str_, regex_mode_);
+    valid_ = true;
+  } catch (const std::regex_error &e) {
+    throw Exception(REGEX_EXCEPTION, e.what());
+  }
+#else
+  compileRegex(compiled_regex_, regex_str_);
+  compileRegex(compiled_full_input_regex_, '^' + regex_str_ + '$');
+  valid_ = true;
+#endif
+}
+
+Regex::Regex(const Regex& other)
+#ifndef NO_MORE_REGFREEE
+  : valid_(false),
+    regex_mode_(REG_EXTENDED)
+#endif
+{
+  *this = other;
+}
+
+Regex& Regex::operator=(const Regex& other) {
+  if (this == &other) {
+    return *this;
+  }
+
+  regex_str_ = other.regex_str_;
+  regex_mode_ = other.regex_mode_;
+#ifdef NO_MORE_REGFREEE
+  compiled_regex_ = other.compiled_regex_;
+#else
+  if (valid_) {
+    regfree(&compiled_regex_);
+    regfree(&compiled_full_input_regex_);
+  }
+  compileRegex(compiled_regex_, regex_str_);
+  compileRegex(compiled_full_input_regex_, '^' + regex_str_ + '$');
+#endif
+  valid_ = other.valid_;
+  return *this;
+}
+
+Regex::Regex(Regex&& other)
+#ifndef NO_MORE_REGFREEE
+  : valid_(false),
+    regex_mode_(REG_EXTENDED)
+#endif
+{
+  *this = std::move(other);
+}
+
+Regex& Regex::operator=(Regex&& other) {
+  if (this == &other) {
+    return *this;
+  }
+
+  regex_str_ = std::move(other.regex_str_);
+  regex_mode_ = other.regex_mode_;
+#ifdef NO_MORE_REGFREEE
+  compiled_regex_ = std::move(other.compiled_regex_);
+#else
+  if (valid_) {
+    regfree(&compiled_regex_);
+    regfree(&compiled_full_input_regex_);
+  }
+  compiled_regex_ = other.compiled_regex_;
+  compiled_full_input_regex_ = other.compiled_full_input_regex_;
+#endif
+  valid_ = other.valid_;
+  other.valid_ = false;
+  return *this;
+}
+
+Regex::~Regex() {
+#ifndef NO_MORE_REGFREEE
+  if (valid_) {
+    regfree(&compiled_regex_);
+    regfree(&compiled_full_input_regex_);
+  }
+#endif
+}
+
+#ifndef NO_MORE_REGFREEE
+void Regex::compileRegex(regex_t& regex, const std::string& regex_string) const {
+  int err_code = regcomp(&regex, regex_string.c_str(), regex_mode_);
+  if (err_code) {
+    const size_t size = regerror(err_code, &regex, nullptr, 0);
+    std::vector<char> msg(size);
+    regerror(err_code, &regex, msg.data(), msg.size());
+    throw Exception(REGEX_EXCEPTION, std::string(msg.begin(), msg.end()));
+  }
+}
+#endif
+
+bool regexSearch(const std::string &pattern, const Regex& regex) {
+  if (!regex.valid_) {
+    return false;
+  }
+#ifdef NO_MORE_REGFREEE
+  return std::regex_search(pattern, regex.compiled_regex_);
+#else
+  std::vector<regmatch_t> match;
+  match.resize(getMaxGroupCountOfRegex(regex.regex_str_));
+  return regexec(&regex.compiled_regex_, pattern.c_str(), match.size(), match.data(), 0) == 0;
+#endif
+}
+
+bool regexSearch(const std::string &pattern, SMatch& match, const Regex& regex) {
+  if (!regex.valid_) {
+    return false;
+  }
+#ifdef NO_MORE_REGFREEE
+  return std::regex_search(pattern, match, regex.compiled_regex_);
+#else
+  match.clear();
+  std::vector<regmatch_t> regmatches;
+  regmatches.resize(getMaxGroupCountOfRegex(regex.regex_str_));
+  bool result = regexec(&regex.compiled_regex_, pattern.c_str(), regmatches.size(), regmatches.data(), 0) == 0;
+  match.pattern_ = pattern;
+  for (const auto& regmatch : regmatches) {
+    match.matches_.push_back(SMatch::Regmatch{regmatch, match.pattern_});
+  }
+  return result;
+#endif
+}
+
+bool regexMatch(const std::string &pattern, const Regex& regex) {
+  if (!regex.valid_) {
+    return false;
+  }
+#ifdef NO_MORE_REGFREEE
+  return std::regex_match(pattern, regex.compiled_regex_);
+#else
+  std::vector<regmatch_t> match;
+  match.resize(getMaxGroupCountOfRegex(regex.regex_str_));
+  return regexec(&regex.compiled_full_input_regex_, pattern.c_str(), match.size(), match.data(), 0) == 0;
+#endif
+}
+
+bool regexMatch(const std::string &pattern, SMatch& match, const Regex& regex) {
+  if (!regex.valid_) {
+    return false;
+  }
+#ifdef NO_MORE_REGFREEE
+  return std::regex_match(pattern, match, regex.compiled_regex_);
+#else
+  match.clear();
+  std::vector<regmatch_t> regmatches;
+  regmatches.resize(getMaxGroupCountOfRegex(regex.regex_str_));
+  bool result = regexec(&regex.compiled_full_input_regex_, pattern.c_str(), regmatches.size(), regmatches.data(), 0) == 0;
+  match.pattern_ = pattern;
+  for (const auto& regmatch : regmatches) {
+    match.matches_.push_back(SMatch::Regmatch{regmatch, match.pattern_});
+  }
+  return result;
+#endif
+}
+
+SMatch getLastRegexMatch(const std::string& pattern, const utils::Regex& regex) {
+#ifdef NO_MORE_REGFREEE
+  auto matches = std::sregex_iterator(pattern.begin(), pattern.end(), regex.compiled_regex_);
+  std::smatch last_match;
+  while (matches != std::sregex_iterator()) {
+    last_match = *matches;
+    matches = std::next(matches);
+  }
+  return last_match;
+#else
+  SMatch search_result;
+  SMatch last_match;
+  auto current_str = pattern;
+  while (regexSearch(current_str, search_result, regex)) {
+    last_match = search_result;
+    current_str = search_result.suffix();
+  }
+
+  auto diff = pattern.size() - last_match.pattern_.size();
+  last_match.pattern_ = pattern;
+  for (auto& match : last_match.matches_) {
+    if (match.match.rm_so >= 0) {
+      match.match.rm_so += diff;
+      match.match.rm_eo += diff;
+    }
+  }
+  return last_match;
+#endif
+}
+
+}  // namespace org::apache::nifi::minifi::utils
diff --git a/libminifi/src/utils/StringUtils.cpp b/libminifi/src/utils/StringUtils.cpp
index a4ae0851c..8913963fb 100644
--- a/libminifi/src/utils/StringUtils.cpp
+++ b/libminifi/src/utils/StringUtils.cpp
@@ -474,16 +474,6 @@ std::string StringUtils::to_base64(const gsl::span<const std::byte> raw_data, bo
   return buf;
 }
 
-std::smatch StringUtils::getLastRegexMatch(const std::string& str, const std::regex& pattern) {
-  auto matches = std::sregex_iterator(str.begin(), str.end(), pattern);
-  std::smatch last_match;
-  while (matches != std::sregex_iterator()) {
-    last_match = *matches;
-    matches = std::next(matches);
-  }
-  return last_match;
-}
-
 std::string StringUtils::escapeUnprintableBytes(gsl::span<const std::byte> data) {
   constexpr const char* hex_digits = "0123456789abcdef";
   std::string result;
diff --git a/libminifi/test/unit/RegexUtilsTests.cpp b/libminifi/test/unit/RegexUtilsTests.cpp
new file mode 100644
index 000000000..efec1124d
--- /dev/null
+++ b/libminifi/test/unit/RegexUtilsTests.cpp
@@ -0,0 +1,129 @@
+/**
+ *
+ * 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 <string>
+#include <vector>
+
+#include "Exception.h"
+#include "utils/RegexUtils.h"
+#include "../TestBase.h"
+#include "../Catch.h"
+
+using org::apache::nifi::minifi::utils::Regex;
+using org::apache::nifi::minifi::Exception;
+namespace minifi = org::apache::nifi::minifi;
+
+TEST_CASE("TestRegexUtils::single_match", "[regex1]") {
+  std::string pat = "Speed limit 130 | Speed limit 80";
+  std::string rgx1 = "Speed limit ([0-9]+)";
+  std::vector<Regex::Mode> mode = {Regex::Mode::ICASE};
+  Regex r1(rgx1, mode);
+  REQUIRE(minifi::utils::regexSearch(pat, r1));
+}
+
+TEST_CASE("TestRegexUtils::invalid_construction", "[regex2]") {
+  std::string pat = "Speed limit 130 | Speed limit 80";
+  std::string rgx1 = "Speed limit ([0-9]+)";
+  std::string rgx2 = "[Invalid)A(F)";
+  std::vector<Regex::Mode> mode = {Regex::Mode::ICASE};
+  Regex r1(rgx1, mode);
+  REQUIRE_THROWS_WITH(Regex(rgx2, mode), Catch::Contains("Regex Operation"));
+}
+
+TEST_CASE("TestRegexUtils::empty_input", "[regex3]") {
+  std::string pat = "";
+  std::string rgx1 = "Speed limit ([0-9]+)";
+  std::string rgx2 = "";
+  std::string rgx3 = "(.*)";
+  std::vector<Regex::Mode> mode = {Regex::Mode::ICASE};
+  Regex r1(rgx1, mode);
+  REQUIRE(!minifi::utils::regexSearch(pat, r1));
+  Regex r2(rgx2, mode);
+  REQUIRE(minifi::utils::regexSearch(pat, r2));
+  REQUIRE(!minifi::utils::regexSearch("LMN", r1));
+  Regex r3(rgx3);
+  REQUIRE(minifi::utils::regexSearch(pat, r3));
+}
+
+TEST_CASE("TestRegexUtils::check_mode", "[regex4]") {
+  std::string pat = "Speed limit 130 | Speed limit 80";
+  std::string rgx1 = "sPeeD limIt ([0-9]+)";
+  Regex r1(rgx1);
+  REQUIRE(!minifi::utils::regexSearch(pat, r1));
+  std::vector<Regex::Mode> mode = {Regex::Mode::ICASE};
+  Regex r2(rgx1, mode);
+  REQUIRE(minifi::utils::regexSearch(pat, r2));
+}
+
+TEST_CASE("TestRegexUtils::regexMatch works correctly", "[matchesFullInput]") {
+  REQUIRE(minifi::utils::regexMatch("", Regex("")) == true);
+  REQUIRE(minifi::utils::regexMatch("input", Regex("")) == false);
+  REQUIRE(minifi::utils::regexMatch("input", Regex(".*")) == true);
+  REQUIRE(minifi::utils::regexMatch("input", Regex("np")) == false);
+  REQUIRE(minifi::utils::regexMatch("input", Regex(".*np.*")) == true);
+  REQUIRE(minifi::utils::regexMatch("input", Regex("(in|out)put")) == true);
+  REQUIRE(minifi::utils::regexMatch("input", Regex("inpu[aeiou]*")) == false);
+}
+
+TEST_CASE("TestRegexUtils::regexSearch works with groups", "[matchesFullInput]") {
+  std::string pat = "Speed limit 130 | Speed limit 80";
+  std::string rgx1 = "Speed limit ([0-9]+)";
+  Regex r1(rgx1);
+  minifi::utils::SMatch matches;
+  REQUIRE(minifi::utils::regexSearch(pat, matches, r1));
+  REQUIRE(matches.size() == 2);
+  REQUIRE(matches[0].str() == "Speed limit 130");
+  REQUIRE(matches[1].str() == "130");
+  REQUIRE(" | Speed limit 80" == matches.suffix().str());
+}
+
+TEST_CASE("TestRegexUtils::regexMatch works with groups", "[matchesFullInput]") {
+  std::string pat = "Speed limit 130 all the way";
+  std::string rgx1 = "Speed limit ([0-9]+) (.*)";
+  Regex r1(rgx1);
+  minifi::utils::SMatch matches;
+  REQUIRE(minifi::utils::regexMatch(pat, matches, r1));
+  REQUIRE(matches.size() == 3);
+  REQUIRE(matches[0].str() == "Speed limit 130 all the way");
+  REQUIRE(matches[1].str() == "130");
+  REQUIRE(matches[2].str() == "all the way");
+  REQUIRE("" == matches.suffix().str());
+}
+
+TEST_CASE("TestRegexUtils::getLastRegexMatch works correctly", "[getLastRegexMatch]") {
+  utils::Regex pattern("<[0-9]+>");
+  {
+    std::string content = "Foo";
+    auto last_match = minifi::utils::getLastRegexMatch(content, pattern);
+    REQUIRE_FALSE(last_match.ready());
+  }
+  {
+    std::string content = "<1> Foo";
+    auto last_match = minifi::utils::getLastRegexMatch(content, pattern);
+    REQUIRE(last_match.ready());
+    CHECK(last_match.length(0) == 3);
+    CHECK(last_match.position(0) == 0);
+  }
+  {
+    std::string content = "<1> Foo<2> Bar<3> Baz<10> Qux";
+    auto last_match = minifi::utils::getLastRegexMatch(content, pattern);
+    REQUIRE(last_match.ready());
+    CHECK(last_match.length(0) == 4);
+    CHECK(last_match.position(0) == 21);
+  }
+}
diff --git a/libminifi/test/unit/StringUtilsTests.cpp b/libminifi/test/unit/StringUtilsTests.cpp
index 799540162..46c4b1573 100644
--- a/libminifi/test/unit/StringUtilsTests.cpp
+++ b/libminifi/test/unit/StringUtilsTests.cpp
@@ -521,29 +521,6 @@ TEST_CASE("StringUtils::removeFramingCharacters works correctly", "[removeFramin
   REQUIRE(utils::StringUtils::removeFramingCharacters("\"\"abba\"\"", '"') == "\"abba\"");
 }
 
-TEST_CASE("StringUtils::getLastRegexMatch works correctly", "[getLastRegexMatch]") {
-  std::regex pattern("<[0-9]+>");
-  {
-    std::string content = "Foo";
-    auto last_match = StringUtils::getLastRegexMatch(content, pattern);
-    REQUIRE_FALSE(last_match.ready());
-  }
-  {
-    std::string content = "<1> Foo";
-    auto last_match = StringUtils::getLastRegexMatch(content, pattern);
-    REQUIRE(last_match.ready());
-    CHECK(last_match.length(0) == 3);
-    CHECK(last_match.position(0) == 0);
-  }
-  {
-    std::string content = "<1> Foo<2> Bar<3> Baz<10> Qux";
-    auto last_match = StringUtils::getLastRegexMatch(content, pattern);
-    REQUIRE(last_match.ready());
-    CHECK(last_match.length(0) == 4);
-    CHECK(last_match.position(0) == 21);
-  }
-}
-
 // ignore terminating \0 character
 template<size_t N>
 gsl::span<const std::byte> from_cstring(const char (&str)[N]) {


[nifi-minifi-cpp] 03/03: MINIFICPP-1810 Provide logs to be read from docker logs

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

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

commit fbf2a349bf10ee36c220eb9f94e1d01ffdd56dd4
Author: Gabor Gyimesi <ga...@gmail.com>
AuthorDate: Tue Apr 26 16:04:46 2022 +0200

    MINIFICPP-1810 Provide logs to be read from docker logs
    
    We log to stderr by default in docker containers from now on and let the docker logging engine take care of the logs.
    
    Closes #1315
    Signed-off-by: Marton Szasz <sz...@apache.org>
---
 .dockerignore                                      |  3 +-
 bin/minifi.sh                                      |  5 +-
 docker/.dockerignore                               | 17 -------
 docker/Dockerfile                                  |  1 +
 docker/conf/minifi-log.properties                  | 56 ++++++++++++++++++++++
 docker/test/integration/minifi/core/Container.py   |  3 --
 .../integration/minifi/core/DockerTestCluster.py   | 22 +--------
 .../integration/minifi/core/MinifiContainer.py     |  3 --
 .../test/integration/minifi/core/NifiContainer.py  |  3 --
 9 files changed, 64 insertions(+), 49 deletions(-)

diff --git a/.dockerignore b/.dockerignore
index 247ebd57c..5d61cbdb8 100644
--- a/.dockerignore
+++ b/.dockerignore
@@ -37,7 +37,8 @@ bin/minifi
 *_repository*
 logs
 **/cmake-build-*
-docker
+docker/*
+!docker/conf
 target
 *.pyc
 *.swp
diff --git a/bin/minifi.sh b/bin/minifi.sh
index fbdea8899..b9ed692f1 100755
--- a/bin/minifi.sh
+++ b/bin/minifi.sh
@@ -211,7 +211,8 @@ case "\$1" in
             exit 0;
         fi
       fi
-      echo running
+      pid=$$
+      echo ${pid} > "${pid_file}"
       exec "\${minifi_executable}"
       ;;
     status)
@@ -339,6 +340,8 @@ case "$1" in
             exit 0;
         fi
       fi
+      pid=$$
+      echo ${pid} > "${pid_file}"
       exec "${minifi_executable}"
       ;;
     status)
diff --git a/docker/.dockerignore b/docker/.dockerignore
deleted file mode 100644
index f6d019fab..000000000
--- a/docker/.dockerignore
+++ /dev/null
@@ -1,17 +0,0 @@
-# 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.
-
-# Place files you want to exclude from the docker build here similar to .gitignore https://docs.docker.com/engine/reference/builder/#dockerignore-file
-DockerBuild.sh
\ No newline at end of file
diff --git a/docker/Dockerfile b/docker/Dockerfile
index 108764bbd..404a87a0a 100644
--- a/docker/Dockerfile
+++ b/docker/Dockerfile
@@ -174,6 +174,7 @@ RUN addgroup -g ${GID} ${USER} && adduser -u ${UID} -D -G ${USER} -g "" ${USER}
 
 # Copy built minifi distribution from builder
 COPY --from=build --chown=${USER}:${USER} ${MINIFI_VERSIONED_HOME} ${MINIFI_HOME}
+COPY --from=build --chown=${USER}:${USER} ${MINIFI_BASE_DIR}/docker/conf/minifi-log.properties ${MINIFI_HOME}/conf/minifi-log.properties
 
 USER ${USER}
 WORKDIR ${MINIFI_HOME}
diff --git a/docker/conf/minifi-log.properties b/docker/conf/minifi-log.properties
new file mode 100644
index 000000000..459db5085
--- /dev/null
+++ b/docker/conf/minifi-log.properties
@@ -0,0 +1,56 @@
+# 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.
+
+#More verbose pattern by default
+#Format details at https://github.com/gabime/spdlog/wiki/3.-Custom-formatting
+spdlog.pattern=[%Y-%m-%d %H:%M:%S.%e] [%n] [%l] %v
+# uncomment to prune package names
+#spdlog.shorten_names=true
+
+#Old format
+#spdlog.pattern=[%Y-%m-%d %H:%M:%S.%e] [minifi log] [%l] %v
+
+#More compact format example
+#spdlog.pattern=[%D %H:%M:%S.%e] [%L] %v
+
+#appender.rolling=rollingappender
+#appender.rolling.directory=${MINIFI_HOME}/logs
+#appender.rolling.file_name=minifi-app.log
+#appender.rolling.max_files=3
+#appender.rolling.max_file_size=5242880
+
+#Other possible appenders
+#appender.stdout=stdout
+appender.stderr=stderr
+#appender.null=null
+## The syslog appender will log using syslog(3) on *nix, and to the Windows Event Log on Windows
+#appender.syslog=syslog
+
+logger.root=INFO,stderr
+
+#Logging configurable by namespace
+logger.org::apache::nifi::minifi=INFO,stderr
+
+#Logging configurable by class fully qualified name
+#logger.org::apache::nifi::minifi::core::logging::LoggerConfiguration=DEBUG
+
+# Log compression #
+## Enables the agent to keep a limited chunk of the application
+## logs in memory in compressed format. Note that due to its
+## compressed nature this could mean more logs than the contents
+## of the log files.
+## Setting any of these to 0 disables the in-memory log compression.
+#compression.cached.log.max.size=8 MB
+#compression.compressed.log.max.size=8 MB
diff --git a/docker/test/integration/minifi/core/Container.py b/docker/test/integration/minifi/core/Container.py
index 700dddb21..2e0c40641 100644
--- a/docker/test/integration/minifi/core/Container.py
+++ b/docker/test/integration/minifi/core/Container.py
@@ -64,6 +64,3 @@ class Container:
 
     def get_startup_finished_log_entry(self):
         raise NotImplementedError()
-
-    def get_log_file_path(self):
-        return None
diff --git a/docker/test/integration/minifi/core/DockerTestCluster.py b/docker/test/integration/minifi/core/DockerTestCluster.py
index 1a8a7e5e4..c29e0baa5 100644
--- a/docker/test/integration/minifi/core/DockerTestCluster.py
+++ b/docker/test/integration/minifi/core/DockerTestCluster.py
@@ -16,7 +16,6 @@
 
 import json
 import logging
-import subprocess
 import sys
 import time
 import os
@@ -69,26 +68,7 @@ class DockerTestCluster(SingleNodeDockerCluster):
             logging.warning('Container segfaulted: %s', container.name)
             self.segfault = True
 
-        log_file_path = self.containers[container_name].get_log_file_path()
-        if not log_file_path:
-            return container.status, container.logs()
-
-        try:
-            if container.status == 'running':
-                app_log_status, app_log = container.exec_run('/bin/sh -c \'cat ' + log_file_path + '\'')
-                if app_log_status == 0:
-                    return container.status, app_log
-            elif container.status == 'exited':
-                log_file_name = container_name + ".log"
-                code = subprocess.run(["docker", "cp", container_name + ":" + log_file_path, log_file_name]).returncode
-                if code == 0:
-                    output = open(log_file_name, 'rb').read()
-                    os.remove(log_file_name)
-                    return container.status, output
-        except Exception:
-            return container.status, None
-
-        return container.status, None
+        return container.status, container.logs()
 
     def __wait_for_app_logs_impl(self, container_name, log_entry, timeout_seconds, count, use_regex):
         wait_start_time = time.perf_counter()
diff --git a/docker/test/integration/minifi/core/MinifiContainer.py b/docker/test/integration/minifi/core/MinifiContainer.py
index 71cb2c96e..17783eec6 100644
--- a/docker/test/integration/minifi/core/MinifiContainer.py
+++ b/docker/test/integration/minifi/core/MinifiContainer.py
@@ -32,9 +32,6 @@ class MinifiContainer(FlowContainer):
     def get_startup_finished_log_entry(self):
         return "Starting Flow Controller"
 
-    def get_log_file_path(self):
-        return MinifiContainer.MINIFI_ROOT + '/logs/minifi-app.log'
-
     def _create_config(self):
         serializer = Minifi_flow_yaml_serializer()
         test_flow_yaml = serializer.serialize(self.start_nodes)
diff --git a/docker/test/integration/minifi/core/NifiContainer.py b/docker/test/integration/minifi/core/NifiContainer.py
index a9ea539b0..0aadad37f 100644
--- a/docker/test/integration/minifi/core/NifiContainer.py
+++ b/docker/test/integration/minifi/core/NifiContainer.py
@@ -37,9 +37,6 @@ class NifiContainer(FlowContainer):
     def get_startup_finished_log_entry(self):
         return "Starting Flow Controller"
 
-    def get_log_file_path(self):
-        return NifiContainer.NIFI_ROOT + '/logs/nifi-app.log'
-
     def __create_config(self):
         serializer = Nifi_flow_xml_serializer()
         test_flow_xml = serializer.serialize(self.start_nodes, NifiContainer.NIFI_VERSION)


[nifi-minifi-cpp] 01/03: MINIFICPP-1802 Do not make EC2 HTTP calls when AWS extension is not used

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

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

commit 1a521323eb2e72d468bf113931570c19302be2f2
Author: Gabor Gyimesi <ga...@gmail.com>
AuthorDate: Tue Apr 26 16:03:20 2022 +0200

    MINIFICPP-1802 Do not make EC2 HTTP calls when AWS extension is not used
    
    Closes #1306
    Signed-off-by: Marton Szasz <sz...@apache.org>
---
 extensions/aws/processors/DeleteS3Object.cpp | 3 ++-
 extensions/aws/processors/FetchS3Object.cpp  | 3 ++-
 extensions/aws/processors/ListS3.cpp         | 4 +++-
 extensions/aws/processors/PutS3Object.cpp    | 3 ++-
 extensions/aws/processors/S3Processor.cpp    | 7 ++++---
 extensions/aws/processors/S3Processor.h      | 2 +-
 6 files changed, 14 insertions(+), 8 deletions(-)

diff --git a/extensions/aws/processors/DeleteS3Object.cpp b/extensions/aws/processors/DeleteS3Object.cpp
index 873346eef..05d4f1468 100644
--- a/extensions/aws/processors/DeleteS3Object.cpp
+++ b/extensions/aws/processors/DeleteS3Object.cpp
@@ -60,7 +60,8 @@ std::optional<aws::s3::DeleteObjectRequestParameters> DeleteS3Object::buildDelet
     const std::shared_ptr<core::ProcessContext> &context,
     const std::shared_ptr<core::FlowFile> &flow_file,
     const CommonProperties &common_properties) const {
-  aws::s3::DeleteObjectRequestParameters params(common_properties.credentials, client_config_);
+  gsl_Expects(client_config_);
+  aws::s3::DeleteObjectRequestParameters params(common_properties.credentials, *client_config_);
   context->getProperty(ObjectKey, params.object_key, flow_file);
   if (params.object_key.empty() && (!flow_file->getAttribute("filename", params.object_key) || params.object_key.empty())) {
     logger_->log_error("No Object Key is set and default object key 'filename' attribute could not be found!");
diff --git a/extensions/aws/processors/FetchS3Object.cpp b/extensions/aws/processors/FetchS3Object.cpp
index 7667d52e2..e6c7abe2e 100644
--- a/extensions/aws/processors/FetchS3Object.cpp
+++ b/extensions/aws/processors/FetchS3Object.cpp
@@ -74,7 +74,8 @@ std::optional<aws::s3::GetObjectRequestParameters> FetchS3Object::buildFetchS3Re
     const std::shared_ptr<core::ProcessContext> &context,
     const std::shared_ptr<core::FlowFile> &flow_file,
     const CommonProperties &common_properties) const {
-  minifi::aws::s3::GetObjectRequestParameters get_object_params(common_properties.credentials, client_config_);
+  gsl_Expects(client_config_);
+  minifi::aws::s3::GetObjectRequestParameters get_object_params(common_properties.credentials, *client_config_);
   get_object_params.bucket = common_properties.bucket;
   get_object_params.requester_pays = requester_pays_;
 
diff --git a/extensions/aws/processors/ListS3.cpp b/extensions/aws/processors/ListS3.cpp
index 61410ed1f..1ffd420c9 100644
--- a/extensions/aws/processors/ListS3.cpp
+++ b/extensions/aws/processors/ListS3.cpp
@@ -99,7 +99,9 @@ void ListS3::onSchedule(const std::shared_ptr<core::ProcessContext> &context, co
   if (!common_properties) {
     throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Required property is not set or invalid");
   }
-  list_request_params_ = std::make_unique<aws::s3::ListRequestParameters>(common_properties->credentials, client_config_);
+
+  gsl_Expects(client_config_);
+  list_request_params_ = std::make_unique<aws::s3::ListRequestParameters>(common_properties->credentials, *client_config_);
   list_request_params_->setClientConfig(common_properties->proxy, common_properties->endpoint_override_url);
   list_request_params_->bucket = common_properties->bucket;
 
diff --git a/extensions/aws/processors/PutS3Object.cpp b/extensions/aws/processors/PutS3Object.cpp
index 937094737..4027b5a6a 100644
--- a/extensions/aws/processors/PutS3Object.cpp
+++ b/extensions/aws/processors/PutS3Object.cpp
@@ -210,7 +210,8 @@ std::optional<aws::s3::PutObjectRequestParameters> PutS3Object::buildPutS3Reques
     const std::shared_ptr<core::ProcessContext> &context,
     const std::shared_ptr<core::FlowFile> &flow_file,
     const CommonProperties &common_properties) const {
-  aws::s3::PutObjectRequestParameters params(common_properties.credentials, client_config_);
+  gsl_Expects(client_config_);
+  aws::s3::PutObjectRequestParameters params(common_properties.credentials, *client_config_);
   params.setClientConfig(common_properties.proxy, common_properties.endpoint_override_url);
   params.bucket = common_properties.bucket;
   params.user_metadata_map = user_metadata_map_;
diff --git a/extensions/aws/processors/S3Processor.cpp b/extensions/aws/processors/S3Processor.cpp
index ff44a8966..3b22ef2f7 100644
--- a/extensions/aws/processors/S3Processor.cpp
+++ b/extensions/aws/processors/S3Processor.cpp
@@ -193,19 +193,20 @@ std::optional<aws::s3::ProxyOptions> S3Processor::getProxy(const std::shared_ptr
 }
 
 void S3Processor::onSchedule(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSessionFactory>& /*sessionFactory*/) {
+  client_config_ = Aws::Client::ClientConfiguration();
   std::string value;
   if (!context->getProperty(Bucket.getName(), value) || value.empty()) {
     throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Bucket property missing or invalid");
   }
 
-  if (!context->getProperty(Region.getName(), client_config_.region) || client_config_.region.empty() || REGIONS.count(client_config_.region) == 0) {
+  if (!context->getProperty(Region.getName(), client_config_->region) || client_config_->region.empty() || REGIONS.count(client_config_->region) == 0) {
     throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Region property missing or invalid");
   }
-  logger_->log_debug("S3Processor: Region [%s]", client_config_.region);
+  logger_->log_debug("S3Processor: Region [%s]", client_config_->region);
 
   if (auto communications_timeout = context->getProperty<core::TimePeriodValue>(CommunicationsTimeout)) {
     logger_->log_debug("S3Processor: Communications Timeout %" PRId64 " ms", communications_timeout->getMilliseconds().count());
-    client_config_.connectTimeoutMs = gsl::narrow<int64_t>(communications_timeout->getMilliseconds().count());
+    client_config_->connectTimeoutMs = gsl::narrow<int64_t>(communications_timeout->getMilliseconds().count());
   } else {
     throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Communications Timeout missing or invalid");
   }
diff --git a/extensions/aws/processors/S3Processor.h b/extensions/aws/processors/S3Processor.h
index c22738f5b..2957d0d0c 100644
--- a/extensions/aws/processors/S3Processor.h
+++ b/extensions/aws/processors/S3Processor.h
@@ -115,7 +115,7 @@ class S3Processor : public core::Processor {
 
   std::shared_ptr<core::logging::Logger> logger_;
   aws::s3::S3Wrapper s3_wrapper_;
-  Aws::Client::ClientConfiguration client_config_;
+  std::optional<Aws::Client::ClientConfiguration> client_config_;
 };
 
 }  // namespace processors