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 2021/11/03 14:32:31 UTC

[nifi-minifi-cpp] 01/02: MINIFICPP-1632 Add RouteText processor

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 c127d8d64d72041842a62778adf68ed2518f5180
Author: Adam Debreceni <ad...@apache.org>
AuthorDate: Wed Nov 3 15:01:43 2021 +0100

    MINIFICPP-1632 Add RouteText processor
    
    Closes #1168
    Signed-off-by: Marton Szasz <sz...@apache.org>
---
 PROCESSORS.md                                      |  36 ++
 README.md                                          |   2 +-
 .../integration/MiNiFi_integration_test_driver.py  |   9 +-
 docker/test/integration/features/routetext.feature |  21 +
 docker/test/integration/minifi/core/utils.py       |  20 +
 .../integration/minifi/processors/RouteText.py     |   9 +
 extensions/http-curl/client/HTTPClient.cpp         |   4 +-
 .../standard-processors/processors/RouteText.cpp   | 476 ++++++++++++++++++++
 .../standard-processors/processors/RouteText.h     | 123 ++++++
 .../tests/unit/RouteTextTests.cpp                  | 485 +++++++++++++++++++++
 libminifi/include/controllers/SSLContextService.h  |   2 +-
 libminifi/include/core/ProcessContext.h            |  19 +-
 libminifi/include/io/BufferStream.h                |   2 +-
 libminifi/include/io/Stream.h                      |   1 +
 libminifi/include/utils/ProcessorConfigUtils.h     |  13 +
 libminifi/include/utils/StringUtils.h              |  40 +-
 libminifi/src/utils/StringUtils.cpp                |  16 +-
 libminifi/test/TestBase.h                          |  19 +
 libminifi/test/unit/ProcessorConfigUtilsTests.cpp  |  62 +++
 libminifi/test/unit/StringUtilsTests.cpp           |  39 +-
 20 files changed, 1374 insertions(+), 24 deletions(-)

diff --git a/PROCESSORS.md b/PROCESSORS.md
index ae96b13..f7ea514 100644
--- a/PROCESSORS.md
+++ b/PROCESSORS.md
@@ -52,6 +52,7 @@
 - [QueryDatabaseTable](#querydatabasetable)
 - [RetryFlowFile](#retryflowfile)
 - [RouteOnAttribute](#routeonattribute)
+- [RouteText](#routetext)
 - [TailFile](#tailfile)
 - [UnfocusArchiveEntry](#unfocusarchiveentry)
 - [UpdateAttribute](#updateattribute)
@@ -1540,6 +1541,41 @@ In the list below, the names of required properties appear in bold. Any other pr
 |failure|Failed files are transferred to failure|
 |unmatched|Files which do not match any expression are routed here|
 
+## RouteText
+
+### Description
+Routes textual data based on a set of user-defined rules. Each segment in an incoming FlowFile is compared against the values specified by user-defined Properties. The mechanism by which the text is compared to these user-defined properties is defined by the 'Matching Strategy'. The data is then routed according to these rules, routing each segment of the text individually.
+
+### Properties
+| Name | Default Value | Allowable Values | Description |
+| - | - | - | - |
+| Grouping Fallback Value |empty string ("")|| If the 'Grouping Regular Expression' is specified and the matching fails, this value will be considered the group of the segment. |
+| Grouping Regular Expression ||| Specifies a Regular Expression to evaluate against each segment to determine which Group it should be placed in. The Regular Expression must have at least one Capturing Group that defines the segment's Group. If multiple Capturing Groups exist in the Regular Expression, the values from all Capturing Groups will be concatenated together. Two segments will not be placed into the same FlowFile unless they both have the same value for the Group (or neither m [...]
+| Ignore Case | false || If true, capitalization will not be taken into account when comparing values. E.g., matching against 'HELLO' or 'hello' will have the same result. This property is ignored if the 'Matching Strategy' is set to 'Satisfies Expression'. |
+| Ignore Leading/Trailing Whitespace | true || Indicates whether or not the whitespace at the beginning and end should be ignored when evaluating a segment. |
+| Matching Strategy | Starts With<br>Ends With<br>Contains<br>Equals<br>Matches Regex<br>Contains Regex<br>Satisfies Expression || Specifies how to evaluate each segment of incoming text against the user-defined properties. |
+| Routing Strategy | Dynamic Routing | Dynamic Routing<br>Route On All<br>Route On Any | Specifies how to determine which Relationship(s) to use when evaluating the segments of incoming text against the 'Matching Strategy' and user-defined properties. 'Dynamic Routing' routes to all the matching dynamic relationships (or 'unmatched' if none matches). 'Route On All' routes to 'matched' iff all dynamic relationships match. 'Route On Any' routes to 'matched' iff any of the dynamic relations [...]
+| Segmentation Strategy | Per Line | Per Line<br>Full Text | Specifies what portions of the FlowFile content constitutes a single segment to be processed. |
+
+### Dynamic Properties
+
+| Name | Value | Description |
+| - | - | - |
+|Relationship Name|value to match against|Routes data that matches the value specified in the Dynamic Property Value to the Relationship specified in the Dynamic Property Key.<br>**Supports Expression Language: true**|
+
+### Relationships
+
+| Name | Description |
+| - | - |
+|matched|Segments that satisfy the required user-defined rules will be routed to this Relationship|
+|unmatched|Segments that do not satisfy the required user-defined rules will be routed to this Relationship|
+|original|The original input file will be routed to this destination|
+
+### Writes Attributes
+
+| Name | Description |
+| - | - |
+|RouteText.Group|The value captured by all capturing groups in the 'Grouping Regular Expression' property. If this property is not set, this attribute will not be added.|
 
 ## TailFile
 
diff --git a/README.md b/README.md
index 3c6a2a9..2ba9e72 100644
--- a/README.md
+++ b/README.md
@@ -66,7 +66,7 @@ The following table lists the base set of processors.
 
 | Extension Set        | Processors           |
 | ------------- |:-------------|
-| **Base**    | [AppendHostInfo](PROCESSORS.md#appendhostinfo)<br/>[DefragmentText](PROCESSORS.md#defragmenttext)<br/>[ExecuteProcess](PROCESSORS.md#executeprocess)<br/>[ExtractText](PROCESSORS.md#extracttext)<br/> [GenerateFlowFile](PROCESSORS.md#generateflowfile)<br/>[GetFile](PROCESSORS.md#getfile)<br/>[GetTCP](PROCESSORS.md#gettcp)<br/>[HashContent](PROCESSORS.md#hashcontent)<br/>[ListenSyslog](PROCESSORS.md#listensyslog)<br/>[LogAttribute](PROCESSORS.md#logattribute)<br/>[PutFile](P [...]
+| **Base**    | [AppendHostInfo](PROCESSORS.md#appendhostinfo)<br/>[DefragmentText](PROCESSORS.md#defragmenttext)<br/>[ExecuteProcess](PROCESSORS.md#executeprocess)<br/>[ExtractText](PROCESSORS.md#extracttext)<br/> [GenerateFlowFile](PROCESSORS.md#generateflowfile)<br/>[GetFile](PROCESSORS.md#getfile)<br/>[GetTCP](PROCESSORS.md#gettcp)<br/>[HashContent](PROCESSORS.md#hashcontent)<br/>[ListenSyslog](PROCESSORS.md#listensyslog)<br/>[LogAttribute](PROCESSORS.md#logattribute)<br/>[PutFile](P [...]
 
 The next table outlines CMAKE flags that correspond with MiNiFi extensions. Extensions that are enabled by default ( such as CURL ), can be disabled with the respective CMAKE flag on the command line.
 
diff --git a/docker/test/integration/MiNiFi_integration_test_driver.py b/docker/test/integration/MiNiFi_integration_test_driver.py
index a1bbb53..b7e5991 100644
--- a/docker/test/integration/MiNiFi_integration_test_driver.py
+++ b/docker/test/integration/MiNiFi_integration_test_driver.py
@@ -18,6 +18,8 @@ from minifi.validators.NoContentCheckFileNumberValidator import NoContentCheckFi
 from minifi.validators.NumFileRangeValidator import NumFileRangeValidator
 from minifi.validators.SingleJSONFileOutputValidator import SingleJSONFileOutputValidator
 
+from minifi.core.utils import decode_escaped_str
+
 
 class MiNiFi_integration_test():
     def __init__(self, image_store):
@@ -104,6 +106,7 @@ class MiNiFi_integration_test():
         return input_port_node
 
     def add_test_data(self, path, test_data, file_name=str(uuid.uuid4())):
+        test_data = decode_escaped_str(test_data)
         self.docker_directory_bindings.put_file_to_docker_path(self.test_id, path, file_name, test_data.encode('utf-8'))
 
     def put_test_resource(self, file_name, contents):
@@ -121,7 +124,7 @@ class MiNiFi_integration_test():
         self.check_output(timeout_seconds, output_validator, 1)
 
     def check_for_single_file_with_content_generated(self, content, timeout_seconds):
-        output_validator = SingleFileOutputValidator(content)
+        output_validator = SingleFileOutputValidator(decode_escaped_str(content))
         output_validator.set_output_dir(self.file_system_observer.get_output_dir())
         self.check_output(timeout_seconds, output_validator, 1)
 
@@ -131,12 +134,12 @@ class MiNiFi_integration_test():
         self.check_output(timeout_seconds, output_validator, 1)
 
     def check_for_multiple_files_generated(self, file_count, timeout_seconds, expected_content=[]):
-        output_validator = MultiFileOutputValidator(file_count, expected_content)
+        output_validator = MultiFileOutputValidator(file_count, [decode_escaped_str(content) for content in expected_content])
         output_validator.set_output_dir(self.file_system_observer.get_output_dir())
         self.check_output(timeout_seconds, output_validator, file_count)
 
     def check_for_at_least_one_file_with_content_generated(self, content, timeout_seconds):
-        output_validator = SingleOrMultiFileOutputValidator(content)
+        output_validator = SingleOrMultiFileOutputValidator(decode_escaped_str(content))
         output_validator.set_output_dir(self.file_system_observer.get_output_dir())
         self.check_output(timeout_seconds, output_validator, 1)
 
diff --git a/docker/test/integration/features/routetext.feature b/docker/test/integration/features/routetext.feature
new file mode 100644
index 0000000..1b82384
--- /dev/null
+++ b/docker/test/integration/features/routetext.feature
@@ -0,0 +1,21 @@
+Feature: Processing log files line-by-line using RouteText
+  Background:
+    Given the content of "/tmp/output" is monitored
+
+  Scenario: Write different level of logs to different files
+    Given a GetFile processor with the "Input Directory" property set to "/tmp/input"
+    And a file with filename "test_file.log" and content "[INFO] one\n[WARNING] two\n[INFO] three\n[WARNING] four\n" is present in "/tmp/input"
+    And a RouteText processor with the "Routing Strategy" property set to "Dynamic Routing"
+    And the "Matching Strategy" property of the RouteText processor is set to "Starts With"
+    And the "Info" property of the RouteText processor is set to "[INFO]"
+    And the "Warning" property of the RouteText processor is set to "[WARNING]"
+    And a UpdateAttribute processor with the name "UpdateInfo" and the "filename" property set to "info.txt"
+    And a UpdateAttribute processor with the name "UpdateWarning" and the "filename" property set to "warning.txt"
+    And a PutFile processor with the "Directory" property set to "/tmp/output"
+    And the "success" relationship of the GetFile processor is connected to the RouteText
+    And the "Info" relationship of the RouteText processor is connected to the UpdateInfo
+    And the "Warning" relationship of the RouteText processor is connected to the UpdateWarning
+    And the "success" relationship of the UpdateInfo processor is connected to the PutFile
+    And the "success" relationship of the UpdateWarning processor is connected to the PutFile
+    When the MiNiFi instance starts up
+    Then two flowfiles with the contents "[INFO] one\n[INFO] three\n" and "[WARNING] two\n[WARNING] four\n" are placed in the monitored directory in less than 10 seconds
diff --git a/docker/test/integration/minifi/core/utils.py b/docker/test/integration/minifi/core/utils.py
index ff44d32..53c4c2b 100644
--- a/docker/test/integration/minifi/core/utils.py
+++ b/docker/test/integration/minifi/core/utils.py
@@ -16,3 +16,23 @@ def retry_check(max_tries=5, retry_interval=1):
             return False
         return retry_wrapper
     return retry_check_func
+
+
+def decode_escaped_str(str):
+    special = {"n": "\n", "v": "\v", "t": "\t", "f": "\f", "r": "\r", "a": "\a", "\\": "\\"}
+    escaped = False
+    result = ""
+    for ch in str:
+        if escaped:
+            if ch in special:
+                result += special[ch]
+            else:
+                result += "\\" + ch
+            escaped = False
+        elif ch == "\\":
+            escaped = True
+        else:
+            result += ch
+    if escaped:
+        result += "\\"
+    return result
diff --git a/docker/test/integration/minifi/processors/RouteText.py b/docker/test/integration/minifi/processors/RouteText.py
new file mode 100644
index 0000000..164fa97
--- /dev/null
+++ b/docker/test/integration/minifi/processors/RouteText.py
@@ -0,0 +1,9 @@
+from ..core.Processor import Processor
+
+
+class RouteText(Processor):
+    def __init__(self, schedule={'scheduling strategy': 'EVENT_DRIVEN'}):
+        super(RouteText, self).__init__(
+            'RouteText',
+            schedule=schedule,
+            auto_terminate=['unmatched', "matched", "original"])
diff --git a/extensions/http-curl/client/HTTPClient.cpp b/extensions/http-curl/client/HTTPClient.cpp
index 1e9c6db..166b67c 100644
--- a/extensions/http-curl/client/HTTPClient.cpp
+++ b/extensions/http-curl/client/HTTPClient.cpp
@@ -383,7 +383,7 @@ void HTTPClient::configure_secure_connection(CURL *http_session) {
   logger_->log_debug("Using CA certificate file \"%s\"", ssl_context_service_->getCACertificate());
 #if 0  // Reenable this path once we change from the direct manipulation of the SSL context to using the cURL API
   if (!ssl_context_service_->getCertificateFile().empty()) {
-    if (utils::StringUtils::endsWithIgnoreCase(ssl_context_service_->getCertificateFile(), "p12")) {
+    if (utils::StringUtils::endsWith(ssl_context_service_->getCertificateFile(), "p12", false)) {
       curl_easy_setopt(http_session, CURLOPT_SSLCERTTYPE, "P12");
     } else {
       curl_easy_setopt(http_session, CURLOPT_SSLCERTTYPE, "PEM");
@@ -391,7 +391,7 @@ void HTTPClient::configure_secure_connection(CURL *http_session) {
     curl_easy_setopt(http_session, CURLOPT_SSLCERT, ssl_context_service_->getCertificateFile().c_str());
   }
   if (!ssl_context_service_->getPrivateKeyFile().empty()) {
-    if (utils::StringUtils::endsWithIgnoreCase(ssl_context_service_->getPrivateKeyFile(), "p12")) {
+    if (utils::StringUtils::endsWith(ssl_context_service_->getPrivateKeyFile(), "p12", false)) {
       curl_easy_setopt(http_session, CURLOPT_SSLKEYTYPE, "P12");
     } else {
       curl_easy_setopt(http_session, CURLOPT_SSLKEYTYPE, "PEM");
diff --git a/extensions/standard-processors/processors/RouteText.cpp b/extensions/standard-processors/processors/RouteText.cpp
new file mode 100644
index 0000000..583fd3b
--- /dev/null
+++ b/extensions/standard-processors/processors/RouteText.cpp
@@ -0,0 +1,476 @@
+/**
+ * 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 "RouteText.h"
+
+#include <map>
+#include <vector>
+#include <utility>
+#include <algorithm>
+#include <set>
+
+#ifdef __APPLE__
+#include <experimental/functional>
+template<typename It, typename Hash, typename Eq>
+using boyer_moore_searcher = std::experimental::boyer_moore_searcher<It, Hash, Eq>;
+#else
+#include <functional>
+template<typename It, typename Hash, typename Eq>
+using boyer_moore_searcher = std::boyer_moore_searcher<It, Hash, Eq>;
+#endif
+
+#include "logging/LoggerConfiguration.h"
+#include "utils/ProcessorConfigUtils.h"
+#include "utils/OptionalUtils.h"
+#include "range/v3/view/transform.hpp"
+#include "range/v3/range/conversion.hpp"
+#include "range/v3/view/tail.hpp"
+#include "range/v3/view/join.hpp"
+#include "range/v3/view/cache1.hpp"
+#include "core/Resource.h"
+
+namespace org::apache::nifi::minifi::processors {
+
+const core::Property RouteText::RoutingStrategy(
+    core::PropertyBuilder::createProperty("Routing Strategy")
+    ->withDescription("Specifies how to determine which Relationship(s) to use when evaluating the segments "
+                      "of incoming text against the 'Matching Strategy' and user-defined properties. "
+                      "'Dynamic Routing' routes to all the matching dynamic relationships (or 'unmatched' if none matches). "
+                      "'Route On All' routes to 'matched' iff all dynamic relationships match. "
+                      "'Route On Any' routes to 'matched' iff any of the dynamic relationships match. ")
+    ->isRequired(true)
+    ->withDefaultValue<std::string>(toString(Routing::DYNAMIC))
+    ->withAllowableValues<std::string>(Routing::values())
+    ->build());
+
+const core::Property RouteText::MatchingStrategy(
+    core::PropertyBuilder::createProperty("Matching Strategy")
+    ->withDescription("Specifies how to evaluate each segment of incoming text against the user-defined properties. "
+                      "Possible values are: 'Starts With', 'Ends With', 'Contains', 'Equals', 'Matches Regex', 'Contains Regex', 'Satisfies Expression'.")
+    ->isRequired(true)
+    ->withAllowableValues<std::string>(Matching::values())
+    ->build());
+
+const core::Property RouteText::TrimWhitespace(
+    core::PropertyBuilder::createProperty("Ignore Leading/Trailing Whitespace")
+    ->withDescription("Indicates whether or not the whitespace at the beginning and end should be ignored when evaluating a segment.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(true)
+    ->build());
+
+const core::Property RouteText::IgnoreCase(
+    core::PropertyBuilder::createProperty("Ignore Case")
+    ->withDescription("If true, capitalization will not be taken into account when comparing values. E.g., matching against 'HELLO' or 'hello' will have the same result. "
+                      "This property is ignored if the 'Matching Strategy' is set to 'Satisfies Expression'.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(false)
+    ->build());
+
+const core::Property RouteText::GroupingRegex(
+    core::PropertyBuilder::createProperty("Grouping Regular Expression")
+    ->withDescription("Specifies a Regular Expression to evaluate against each segment to determine which Group it should be placed in. "
+                      "The Regular Expression must have at least one Capturing Group that defines the segment's Group. If multiple Capturing Groups "
+                      "exist in the Regular Expression, the values from all Capturing Groups will be joined together with \", \". Two segments will not be "
+                      "placed into the same FlowFile unless they both have the same value for the Group (or neither matches the Regular Expression). "
+                      "For example, to group together all lines in a CSV File by the first column, we can set this value to \"(.*?),.*\" (and use \"Per Line\" segmentation). "
+                      "Two segments that have the same Group but different Relationships will never be placed into the same FlowFile.")
+    ->build());
+
+const core::Property RouteText::GroupingFallbackValue(
+    core::PropertyBuilder::createProperty("Grouping Fallback Value")
+    ->withDescription("If the 'Grouping Regular Expression' is specified and the matching fails, this value will be considered the group of the segment.")
+    ->withDefaultValue<std::string>("")
+    ->build());
+
+const core::Property RouteText::SegmentationStrategy(
+    core::PropertyBuilder::createProperty("Segmentation Strategy")
+    ->withDescription("Specifies what portions of the FlowFile content constitutes a single segment to be processed. "
+                      "'Full Text' considers the whole content as a single segment, 'Per Line' considers each line of the content as a separate segment")
+    ->isRequired(true)
+    ->withDefaultValue<std::string>(toString(Segmentation::PER_LINE))
+    ->withAllowableValues<std::string>(Segmentation::values())
+    ->build());
+
+const core::Relationship RouteText::Original("original", "The original input file will be routed to this destination");
+
+const core::Relationship RouteText::Unmatched("unmatched", "Segments that do not satisfy the required user-defined rules will be routed to this Relationship");
+
+const core::Relationship RouteText::Matched("matched", "Segments that satisfy the required user-defined rules will be routed to this Relationship");
+
+RouteText::RouteText(const std::string& name, const utils::Identifier& uuid)
+    : core::Processor(name, uuid), logger_(core::logging::LoggerFactory<RouteText>::getLogger()) {}
+
+void RouteText::initialize() {
+  setSupportedProperties({
+     RoutingStrategy,
+     MatchingStrategy,
+     TrimWhitespace,
+     IgnoreCase,
+     GroupingRegex,
+     GroupingFallbackValue,
+     SegmentationStrategy
+  });
+  setSupportedRelationships({Original, Unmatched, Matched});
+}
+
+void RouteText::onSchedule(core::ProcessContext* context, core::ProcessSessionFactory* /*sessionFactory*/) {
+  gsl_Expects(context);
+  routing_ = utils::parseEnumProperty<Routing>(*context, RoutingStrategy);
+  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);});
+  segmentation_ = utils::parseEnumProperty<Segmentation>(*context, SegmentationStrategy);
+  context->getProperty(GroupingFallbackValue.getName(), group_fallback_);
+}
+
+class RouteText::ReadCallback : public InputStreamCallback {
+  using Fn = std::function<void(Segment)>;
+
+ public:
+  ReadCallback(Segmentation segmentation, size_t file_size, Fn&& fn)
+    : segmentation_(segmentation), file_size_(file_size), fn_(std::move(fn)) {}
+
+  int64_t process(const std::shared_ptr<io::BaseStream>& stream) override {
+    std::vector<uint8_t> buffer;
+    size_t ret = stream->read(buffer, file_size_);
+    if (io::isError(ret)) {
+      return -1;
+    }
+    if (ret != file_size_) {
+      throw Exception(PROCESS_SESSION_EXCEPTION, "Couldn't read whole flowfile content");
+    }
+    std::string_view content{reinterpret_cast<const char*>(buffer.data()), buffer.size()};
+    switch (segmentation_.value()) {
+      case Segmentation::FULL_TEXT: {
+        fn_({content, 0});
+        return content.length();
+      }
+      case Segmentation::PER_LINE: {
+        // 1-based index as in nifi
+        size_t segment_idx = 1;
+        std::string_view::size_type curr = 0;
+        while (curr < content.length()) {
+          // find beginning of next line
+          std::string_view::size_type next_line = content.find('\n', curr);
+
+          if (next_line == std::string_view::npos) {
+            fn_({content.substr(curr), segment_idx});
+          } else {
+            // include newline character to be in-line with nifi semantics
+            ++next_line;
+            fn_({content.substr(curr, next_line - curr), segment_idx});
+          }
+          curr = next_line;
+          ++segment_idx;
+        }
+        return content.length();
+      }
+    }
+    throw Exception(PROCESSOR_EXCEPTION, "Unknown segmentation strategy");
+  }
+
+ private:
+  Segmentation segmentation_;
+  size_t file_size_;
+  Fn fn_;
+};
+
+class RouteText::MatchingContext {
+  struct CaseAwareHash {
+    explicit CaseAwareHash(CasePolicy policy): policy_(policy) {}
+    size_t operator()(char ch) const {
+      if (policy_ == CasePolicy::CASE_SENSITIVE) {
+        return static_cast<size_t>(ch);
+      }
+      return std::hash<int>{}(std::tolower(static_cast<unsigned char>(ch)));
+    }
+
+   private:
+    CasePolicy policy_;
+  };
+
+  struct CaseAwareEq {
+    explicit CaseAwareEq(CasePolicy policy): policy_(policy) {}
+    bool operator()(char a, char b) const {
+      if (policy_ == CasePolicy::CASE_SENSITIVE) {
+        return a == b;
+      }
+      return std::tolower(static_cast<unsigned char>(a)) == std::tolower(static_cast<unsigned char>(b));
+    }
+
+   private:
+    CasePolicy policy_;
+  };
+  using Searcher = boyer_moore_searcher<std::string::const_iterator, CaseAwareHash, CaseAwareEq>;
+
+ public:
+  MatchingContext(core::ProcessContext& process_context, std::shared_ptr<core::FlowFile> flow_file, CasePolicy case_policy)
+    : process_context_(process_context),
+      flow_file_(std::move(flow_file)),
+      case_policy_(case_policy) {}
+
+  const std::regex& getRegexProperty(const core::Property& prop) {
+    auto it = regex_values_.find(prop.getName());
+    if (it != regex_values_.end()) {
+      return it->second;
+    }
+    std::string value;
+    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;
+    if (case_policy_ == CasePolicy::IGNORE_CASE) {
+      flags |= std::regex::icase;
+    }
+    return (regex_values_[prop.getName()] = std::regex(value, flags));
+  }
+
+  const std::string& getStringProperty(const core::Property& prop) {
+    auto it = string_values_.find(prop.getName());
+    if (it != string_values_.end()) {
+      return it->second;
+    }
+    std::string value;
+    if (!process_context_.getDynamicProperty(prop, value, flow_file_)) {
+      throw Exception(PROCESSOR_EXCEPTION, "Missing dynamic property: '" + prop.getName() + "'");
+    }
+    return (string_values_[prop.getName()] = value);
+  }
+
+  const Searcher& getSearcher(const core::Property& prop) {
+    auto it = searcher_values_.find(prop.getName());
+    if (it != searcher_values_.end()) {
+      return it->second.searcher_;
+    }
+    std::string value;
+    if (!process_context_.getDynamicProperty(prop, value, flow_file_)) {
+      throw Exception(PROCESSOR_EXCEPTION, "Missing dynamic property: '" + prop.getName() + "'");
+    }
+
+    return searcher_values_.emplace(
+        std::piecewise_construct, std::forward_as_tuple(prop.getName()),
+        std::forward_as_tuple(value, case_policy_)).first->second.searcher_;
+  }
+
+  core::ProcessContext& process_context_;
+  std::shared_ptr<core::FlowFile> flow_file_;
+  CasePolicy case_policy_;
+
+  std::map<std::string, std::string> string_values_;
+  std::map<std::string, std::regex> regex_values_;
+
+  struct OwningSearcher {
+    OwningSearcher(std::string str, CasePolicy case_policy)
+      : str_(std::move(str)), searcher_(str_.cbegin(), str_.cend(), CaseAwareHash{case_policy}, CaseAwareEq{case_policy}) {}
+    OwningSearcher(const OwningSearcher&) = delete;
+    OwningSearcher(OwningSearcher&&) = delete;
+    OwningSearcher& operator=(const OwningSearcher&) = delete;
+    OwningSearcher& operator=(OwningSearcher&&) = delete;
+
+    std::string str_;
+    Searcher searcher_;
+  };
+
+  std::map<std::string, OwningSearcher> searcher_values_;
+};
+
+namespace {
+struct Route {
+  core::Relationship relationship_;
+  std::optional<std::string> group_name_;
+
+  bool operator<(const Route& other) const {
+    return std::tie(relationship_, group_name_) < std::tie(other.relationship_, other.group_name_);
+  }
+};
+}  // namespace
+
+void RouteText::onTrigger(core::ProcessContext *context, core::ProcessSession *session) {
+  gsl_Expects(context && session);
+  auto flow_file = session->get();
+  if (!flow_file) {
+    context->yield();
+    return;
+  }
+
+  std::map<Route, std::string> flow_file_contents;
+
+  MatchingContext matching_context(*context, flow_file, case_policy_);
+
+  ReadCallback callback(segmentation_, flow_file->getSize(), [&] (Segment segment) {
+    std::string_view original_value = segment.value_;
+    std::string_view preprocessed_value = preprocess(segment.value_);
+
+    if (matching_ != Matching::EXPRESSION) {
+      // an Expression has access to the raw segment like in nifi
+      // all others use the preprocessed_value
+      segment.value_ = preprocessed_value;
+    }
+
+    // group extraction always uses the preprocessed
+    auto group = getGroup(preprocessed_value);
+    switch (routing_.value()) {
+      case Routing::ALL: {
+        if (std::all_of(dynamic_properties_.cbegin(), dynamic_properties_.cend(), [&] (const auto& prop) {
+          return matchSegment(matching_context, segment, prop.second);
+        })) {
+          flow_file_contents[{Matched, group}] += original_value;
+        } else {
+          flow_file_contents[{Unmatched, group}] += original_value;
+        }
+        return;
+      }
+      case Routing::ANY: {
+        if (std::any_of(dynamic_properties_.cbegin(), dynamic_properties_.cend(), [&] (const auto& prop) {
+          return matchSegment(matching_context, segment, prop.second);
+        })) {
+          flow_file_contents[{Matched, group}] += original_value;
+        } else {
+          flow_file_contents[{Unmatched, group}] += original_value;
+        }
+        return;
+      }
+      case Routing::DYNAMIC: {
+        bool routed = false;
+        for (const auto& [property_name, prop] : dynamic_properties_) {
+          if (matchSegment(matching_context, segment, prop)) {
+            flow_file_contents[{dynamic_relationships_[property_name], group}] += original_value;
+            routed = true;
+          }
+        }
+        if (!routed) {
+          flow_file_contents[{Unmatched, group}] += original_value;
+        }
+        return;
+      }
+    }
+    throw Exception(PROCESSOR_EXCEPTION, "Unknown routing strategy");
+  });
+  session->read(flow_file, &callback);
+
+  for (const auto& [route, content] : flow_file_contents) {
+    auto new_flow_file = session->create(flow_file);
+    if (route.group_name_) {
+      new_flow_file->setAttribute(GROUP_ATTRIBUTE_NAME, route.group_name_.value());
+    }
+    session->writeBuffer(new_flow_file, content);
+    session->transfer(new_flow_file, route.relationship_);
+  }
+
+  session->transfer(flow_file, Original);
+}
+
+std::string_view RouteText::preprocess(std::string_view str) const {
+  if (segmentation_ == Segmentation::PER_LINE) {
+    // do not consider the trailing \r\n characters in order to conform to nifi
+    auto len = str.find_last_not_of("\r\n");
+    if (len != std::string_view::npos) {
+      str = str.substr(0, len + 1);
+    } else {
+      str = "";
+    }
+  }
+  if (trim_) {
+    str = utils::StringUtils::trim(str);
+  }
+  return str;
+}
+
+bool RouteText::matchSegment(MatchingContext& context, const Segment& segment, const core::Property& prop) const {
+  switch (matching_.value()) {
+    case Matching::EXPRESSION: {
+      std::map<std::string, std::string> variables;
+      variables["segment"] = segment.value_;
+      variables["segmentNo"] = std::to_string(segment.idx_);
+      if (segmentation_ == Segmentation::PER_LINE) {
+        // for nifi compatibility
+        variables["line"] = segment.value_;
+        variables["lineNo"] = std::to_string(segment.idx_);
+      }
+      std::string result;
+      if (context.process_context_.getDynamicProperty(prop, result, context.flow_file_, variables)) {
+        return utils::StringUtils::toBool(result).value_or(false);
+      } else {
+        throw Exception(PROCESSOR_EXCEPTION, "Missing dynamic property: '" + prop.getName() + "'");
+      }
+    }
+    case Matching::STARTS_WITH: {
+      return utils::StringUtils::startsWith(segment.value_, context.getStringProperty(prop), case_policy_ == CasePolicy::CASE_SENSITIVE);
+    }
+    case Matching::ENDS_WITH: {
+      return utils::StringUtils::endsWith(segment.value_, context.getStringProperty(prop), case_policy_ == CasePolicy::CASE_SENSITIVE);
+    }
+    case Matching::CONTAINS: {
+      return std::search(segment.value_.begin(), segment.value_.end(), context.getSearcher(prop)) != segment.value_.end();
+    }
+    case Matching::EQUALS: {
+      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));
+    }
+    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));
+    }
+  }
+  throw Exception(PROCESSOR_EXCEPTION, "Unknown matching strategy");
+}
+
+std::optional<std::string> RouteText::getGroup(const std::string_view& segment) const {
+  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())) {
+    return group_fallback_;
+  }
+  // WARNING!! using a temporary std::string causes the omission of delimiters
+  // in the output on Windows
+  const std::string comma = ", ";
+  // unused capturing groups default to empty string
+  auto to_string = [] (const auto& submatch) -> std::string {return submatch;};
+  return ranges::views::tail(match_result)  // only join the capture groups
+    | ranges::views::transform(to_string)
+    | ranges::views::cache1
+    | ranges::views::join(comma)
+    | ranges::to<std::string>();
+}
+
+void RouteText::onDynamicPropertyModified(const core::Property& /*orig_property*/, const core::Property& new_property) {
+  dynamic_properties_[new_property.getName()] = new_property;
+
+  std::set<core::Relationship> relationships{Original, Unmatched, Matched};
+
+  for (const auto& [property_name, prop] : dynamic_properties_) {
+    core::Relationship rel{property_name, "Dynamic Route"};
+    dynamic_relationships_[property_name] = rel;
+    relationships.insert(rel);
+    logger_->log_info("RouteText registered dynamic route '%s' with expression '%s'", property_name, prop.getValue().to_string());
+  }
+
+  setSupportedRelationships(relationships);
+}
+
+REGISTER_RESOURCE(RouteText, "Routes textual data based on a set of user-defined rules. Each segment in an incoming FlowFile is "
+                             "compared against the values specified by user-defined Properties. The mechanism by which the text is compared "
+                             "to these user-defined properties is defined by the 'Matching Strategy'. The data is then routed according to "
+                             "these rules, routing each segment of the text individually.");
+
+}  // namespace org::apache::nifi::minifi::processors
diff --git a/extensions/standard-processors/processors/RouteText.h b/extensions/standard-processors/processors/RouteText.h
new file mode 100644
index 0000000..b30dbad
--- /dev/null
+++ b/extensions/standard-processors/processors/RouteText.h
@@ -0,0 +1,123 @@
+/**
+ * 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 <regex>
+#include <optional>
+#include <string_view>
+#include <map>
+#include <string>
+#include <memory>
+
+#include "Processor.h"
+#include "utils/Enum.h"
+#include "utils/Export.h"
+
+namespace org::apache::nifi::minifi::processors {
+
+class RouteText : public core::Processor {
+ public:
+  EXTENSIONAPI static const core::Property RoutingStrategy;
+  EXTENSIONAPI static const core::Property MatchingStrategy;
+  EXTENSIONAPI static const core::Property TrimWhitespace;
+  EXTENSIONAPI static const core::Property IgnoreCase;
+  EXTENSIONAPI static const core::Property GroupingRegex;
+  EXTENSIONAPI static const core::Property GroupingFallbackValue;
+  EXTENSIONAPI static const core::Property SegmentationStrategy;
+
+  EXTENSIONAPI static const core::Relationship Original;
+  EXTENSIONAPI static const core::Relationship Unmatched;
+  EXTENSIONAPI static const core::Relationship Matched;
+
+  explicit RouteText(const std::string& name, const utils::Identifier& uuid = {});
+
+  void initialize() override;
+  void onSchedule(core::ProcessContext *context, core::ProcessSessionFactory* sessionFactory) override;
+  void onTrigger(core::ProcessContext *context, core::ProcessSession *session) override;
+
+  core::annotation::Input getInputRequirement() const override {
+    return core::annotation::Input::INPUT_REQUIRED;
+  }
+
+  bool supportsDynamicProperties() override {
+    return true;
+  }
+
+  bool supportsDynamicRelationships() override {
+    return true;
+  }
+
+  void onDynamicPropertyModified(const core::Property& orig_property, const core::Property& new_property) override;
+
+ private:
+  static constexpr const char* GROUP_ATTRIBUTE_NAME = "RouteText.Group";
+
+  SMART_ENUM(Routing,
+    (DYNAMIC, "Dynamic Routing"),
+    (ALL, "Route On All"),
+    (ANY, "Route On Any")
+  )
+
+  SMART_ENUM(Matching,
+    (STARTS_WITH, "Starts With"),
+    (ENDS_WITH, "Ends With"),
+    (CONTAINS, "Contains"),
+    (EQUALS, "Equals"),
+    (MATCHES_REGEX, "Matches Regex"),
+    (CONTAINS_REGEX, "Contains Regex"),
+    (EXPRESSION, "Satisfies Expression")
+  )
+
+  SMART_ENUM(Segmentation,
+    (FULL_TEXT, "Full Text"),
+    (PER_LINE, "Per Line")
+  )
+
+  enum class CasePolicy {
+    CASE_SENSITIVE,
+    IGNORE_CASE
+  };
+
+  class ReadCallback;
+
+  class MatchingContext;
+
+  struct Segment {
+    std::string_view value_;
+    size_t idx_;  // 1-based index as in nifi
+  };
+
+  std::string_view preprocess(std::string_view str) const;
+  bool matchSegment(MatchingContext& context, const Segment& segment, const core::Property& prop) const;
+  std::optional<std::string> getGroup(const std::string_view& segment) const;
+
+  Routing routing_;
+  Matching matching_;
+  Segmentation segmentation_;
+  bool trim_{true};
+  CasePolicy case_policy_{CasePolicy::CASE_SENSITIVE};
+  std::optional<std::regex> group_regex_;
+  std::string group_fallback_;
+
+  std::map<std::string, core::Property> dynamic_properties_;
+  std::map<std::string, core::Relationship> dynamic_relationships_;
+
+  std::shared_ptr<core::logging::Logger> logger_;
+};
+
+}  // namespace org::apache::nifi::minifi::processors
diff --git a/extensions/standard-processors/tests/unit/RouteTextTests.cpp b/extensions/standard-processors/tests/unit/RouteTextTests.cpp
new file mode 100644
index 0000000..64aa06f
--- /dev/null
+++ b/extensions/standard-processors/tests/unit/RouteTextTests.cpp
@@ -0,0 +1,485 @@
+/**
+ *
+ * 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 "TestBase.h"
+#include "processors/RouteText.h"
+
+namespace processors = minifi::processors;
+
+struct RouteTextController : public TestController {
+  struct FlowFilePattern {
+    FlowFilePattern& attr(const std::string& name, const std::optional<std::string>& value) {
+      required_attributes_[name] = value;
+      return *this;
+    }
+    FlowFilePattern& content(const std::string& content) {
+      required_content_ = content;
+      return *this;
+    }
+
+    std::map<std::string, std::optional<std::string>> required_attributes_;
+    std::optional<std::string> required_content_;
+  };
+
+  struct FlowFilePatternVec : std::vector<FlowFilePattern> {
+    FlowFilePatternVec() = default;
+    FlowFilePatternVec(std::initializer_list<std::string> args) {
+      for (const auto& arg : args) {
+        push_back(FlowFilePattern().content(arg));
+      }
+    }
+    FlowFilePatternVec(std::initializer_list<FlowFilePattern> patterns) {
+      for (const auto& pattern : patterns) {
+        push_back(pattern);
+      }
+    }
+  };
+
+  RouteTextController() {
+    plan_ = createPlan();
+    plan_->addProcessor(proc_, "RouteText");
+    input_ = plan_->addConnection(nullptr, {"success", ""}, proc_);
+    createOutput(processors::RouteText::Original);
+    createOutput(processors::RouteText::Unmatched);
+    createOutput(processors::RouteText::Matched);
+  }
+
+  void createOutput(const core::Relationship& rel) {
+    outputs_[rel.getName()] = plan_->addConnection(proc_, rel, nullptr);
+  }
+
+  void verifyOutputRelationship(const std::string& rel_name, const FlowFilePatternVec& patterns) {
+    size_t pattern_idx = 0;
+    std::set<std::shared_ptr<core::FlowFile>> expired;
+    while (auto flow_file = outputs_.at(rel_name)->poll(expired)) {
+      REQUIRE(expired.empty());
+      // more flowfiles than patterns
+      REQUIRE(pattern_idx < patterns.size());
+      const auto& pattern = patterns[pattern_idx++];
+      for (const auto& attr : pattern.required_attributes_) {
+        auto attr_value = flow_file->getAttribute(attr.first);
+        REQUIRE(attr_value == attr.second);
+      }
+      if (pattern.required_content_) {
+        REQUIRE(pattern.required_content_.value() == plan_->getContent(flow_file));
+      }
+    }
+    // must use all patterns
+    REQUIRE(pattern_idx == patterns.size());
+  }
+
+  void verifyAllOutput(const std::map<std::string, FlowFilePatternVec>& patterns) {
+    FlowFilePatternVec all;
+    for (const auto& [rel, files] : patterns) {
+      for (const auto& file : files) {
+        all.push_back(file);
+      }
+      verifyOutputRelationship(rel, files);
+    }
+    if (patterns.find("original") == patterns.end()) {
+      // expectations on "original" were implicit
+      verifyOutputRelationship("original", all);
+    }
+  }
+
+  void run() {
+    while (!input_->isEmpty()) {
+      plan_->runProcessor(proc_);
+    }
+  }
+
+  void putFlowFile(const std::map<std::string, std::string>& attributes, const std::string& content) {
+    auto flow_file = std::make_shared<minifi::FlowFileRecord>();
+    for (const auto& attr : attributes) {
+      flow_file->setAttribute(attr.first, attr.second);
+    }
+    auto content_session = plan_->getContentRepo()->createSession();
+    auto claim = content_session->create();
+    auto stream = content_session->write(claim);
+    stream->write(reinterpret_cast<const uint8_t*>(content.c_str()), content.length());
+    flow_file->setResourceClaim(claim);
+    flow_file->setSize(stream->size());
+    flow_file->setOffset(0);
+
+    stream->close();
+    content_session->commit();
+    input_->put(flow_file);
+  }
+
+  std::shared_ptr<TestPlan> plan_ = createPlan();
+  std::shared_ptr<core::Processor> proc_ = std::make_shared<processors::RouteText>("RouteText");
+  std::map<std::string, std::shared_ptr<minifi::Connection>> outputs_;
+  std::shared_ptr<minifi::Connection> input_;
+};
+
+TEST_CASE_METHOD(RouteTextController, "RouteText correctly handles Matching Strategies") {
+  proc_->setProperty(processors::RouteText::RoutingStrategy, "Dynamic Routing");
+
+  std::map<std::string, FlowFilePatternVec> expected{
+      {"here", {}},
+      {"matched", {}},
+      {"unmatched", {}}
+  };
+
+  SECTION("Starts With") {
+    proc_->setProperty(processors::RouteText::MatchingStrategy, "Starts With");
+    proc_->setDynamicProperty("here", "se");
+    SECTION("Case sensitive") {
+      expected["here"] = {"seven"};
+      expected["unmatched"] = {"Seven", "even"};
+    }
+    SECTION("Case insensitive") {
+      proc_->setProperty(processors::RouteText::IgnoreCase, "true");
+      expected["here"] = {"seven", "Seven"};
+      expected["unmatched"] = {"even"};
+    }
+  }
+  SECTION("Ends With") {
+    proc_->setProperty(processors::RouteText::MatchingStrategy, "Ends With");
+    proc_->setDynamicProperty("here", "ven");
+    SECTION("Case sensitive") {
+      expected["here"] = {"seven"};
+      expected["unmatched"] = {"SeveN", "seten"};
+    }
+    SECTION("Case insensitive") {
+      proc_->setProperty(processors::RouteText::IgnoreCase, "true");
+      expected["here"] = {"seven", "SeveN"};
+      expected["unmatched"] = {"seten"};
+    }
+  }
+  SECTION("Contains") {
+    proc_->setProperty(processors::RouteText::MatchingStrategy, "Contains");
+    proc_->setDynamicProperty("here", "eve");
+    SECTION("Case sensitive") {
+      expected["here"] = {"seven"};
+      expected["unmatched"] = {"SeVeN", "seren"};
+    }
+    SECTION("Case insensitive") {
+      proc_->setProperty(processors::RouteText::IgnoreCase, "true");
+      expected["here"] = {"seven", "SeVeN"};
+      expected["unmatched"] = {"seren"};
+    }
+  }
+  SECTION("Equals") {
+    proc_->setProperty(processors::RouteText::MatchingStrategy, "Equals");
+    proc_->setDynamicProperty("here", "seven");
+    SECTION("Case sensitive") {
+      expected["here"] = {"seven"};
+      expected["unmatched"] = {"Seven", "seven1"};
+    }
+    SECTION("Case insensitive") {
+      proc_->setProperty(processors::RouteText::IgnoreCase, "true");
+      expected["here"] = {"seven", "Seven"};
+      expected["unmatched"] = {"seven1"};
+    }
+  }
+  SECTION("Matches Regex") {
+    proc_->setProperty(processors::RouteText::MatchingStrategy, "Matches Regex");
+    proc_->setDynamicProperty("here", "se.en");
+    SECTION("Case sensitive") {
+      expected["here"] = {"seven"};
+      expected["unmatched"] = {"Seven", "sevena"};
+    }
+    SECTION("Case insensitive") {
+      proc_->setProperty(processors::RouteText::IgnoreCase, "true");
+      expected["here"] = {"seven", "Seven"};
+      expected["unmatched"] = {"sevena"};
+    }
+  }
+  SECTION("Contains Regex") {
+    proc_->setProperty(processors::RouteText::MatchingStrategy, "Contains Regex");
+    proc_->setDynamicProperty("here", ".ve");
+    SECTION("Case sensitive") {
+      expected["here"] = {"seven"};
+      expected["unmatched"] = {"SeVeN", "ven"};
+    }
+    SECTION("Case insensitive") {
+      proc_->setProperty(processors::RouteText::IgnoreCase, "true");
+      expected["here"] = {"seven", "SeVeN"};
+      expected["unmatched"] = {"ven"};
+    }
+  }
+  SECTION("Satisfies Expression") {
+    proc_->setProperty(processors::RouteText::MatchingStrategy, "Satisfies Expression");
+    proc_->setDynamicProperty("here", "${segment:equals('seven')}");
+    // case sensitivity does not matter here
+    SECTION("Case sensitive") {
+      expected["here"] = {"seven"};
+      expected["unmatched"] = {"SeVeN", "ven"};
+    }
+    SECTION("Case insensitive matching does not apply in Satisfy Expression mode") {
+      proc_->setProperty(processors::RouteText::IgnoreCase, "true");
+      expected["here"] = {"seven"};
+      expected["unmatched"] = {"SeVeN", "ven"};
+    }
+  }
+
+  createOutput({"here", ""});
+
+  for (const auto& route : expected) {
+    for (const auto& ff : route.second) {
+      putFlowFile({}, ff.required_content_.value());
+    }
+  }
+
+  run();
+
+  verifyAllOutput(expected);
+}
+
+TEST_CASE_METHOD(RouteTextController, "RouteText correctly handles Routing Strategies") {
+  proc_->setProperty(processors::RouteText::MatchingStrategy, "Contains");
+  proc_->setDynamicProperty("one", "apple");
+  proc_->setDynamicProperty("two", "banana");
+
+  createOutput({"one", ""});
+  createOutput({"two", ""});
+
+  std::map<std::string, FlowFilePatternVec> expected{
+      {"one", {}},
+      {"two", {}},
+      {"matched", {}},
+      {"unmatched", {}}
+  };
+
+  SECTION("Dynamic Routing") {
+    proc_->setProperty(processors::RouteText::RoutingStrategy, "Dynamic Routing");
+
+    expected["one"] = {"apple"};
+    expected["two"] = {"banana"};
+    expected["unmatched"] = {"other"};
+  }
+  SECTION("Route On All") {
+    proc_->setProperty(processors::RouteText::RoutingStrategy, "Route On All");
+
+    expected["matched"] = {"apple-banana"};
+    expected["unmatched"] = {"apple", "none"};
+  }
+  SECTION("Route On Any") {
+    proc_->setProperty(processors::RouteText::RoutingStrategy, "Route On Any");
+
+    expected["matched"] = {"apple", "banana", "apple-banana"};
+    expected["unmatched"] = {"none"};
+  }
+
+  for (const auto& route : expected) {
+    for (const auto& ff : route.second) {
+      putFlowFile({}, ff.required_content_.value());
+    }
+  }
+
+  run();
+
+  verifyAllOutput(expected);
+}
+
+TEST_CASE_METHOD(RouteTextController, "RouteText 'Per Line' segmentation") {
+  proc_->setProperty(processors::RouteText::SegmentationStrategy, "Per Line");
+  proc_->setProperty(processors::RouteText::MatchingStrategy, "Equals");
+  proc_->setDynamicProperty("A", "A");
+  proc_->setDynamicProperty("B", "B");
+
+  createOutput({"A", ""});
+  createOutput({"B", ""});
+
+  std::string content = "A\nB\r\n A \r\n\r\rA";
+  putFlowFile({}, content);
+
+  std::map<std::string, FlowFilePatternVec> expected{
+      {"A", {}},
+      {"B", {}},
+      {"matched", {}},
+      {"unmatched", {}},
+      {"original", {content}}
+  };
+
+
+  SECTION("Without trim") {
+    proc_->setProperty(processors::RouteText::TrimWhitespace, "false");
+    expected["A"] = {"A\n"};
+    expected["B"] = {"B\r\n"};
+    expected["unmatched"] = {" A \r\n\r\rA"};
+  }
+
+  SECTION("With trim") {
+    proc_->setProperty(processors::RouteText::TrimWhitespace, "true");
+    expected["A"] = {"A\n A \r\n\r\rA"};
+    expected["B"] = {"B\r\n"};
+  }
+
+  run();
+
+  verifyAllOutput(expected);
+}
+
+TEST_CASE_METHOD(RouteTextController, "RouteText 'Per Line' segmentation ignores trailing empty line") {
+  proc_->setProperty(processors::RouteText::SegmentationStrategy, "Per Line");
+  proc_->setProperty(processors::RouteText::RoutingStrategy, "Route On All");
+  proc_->setProperty(processors::RouteText::MatchingStrategy, "Equals");
+  proc_->setDynamicProperty("A", "A");
+
+  std::string content;
+
+  SECTION("Windows line ending") {
+    content = "A\r\n";
+  }
+  SECTION("Simple line ending") {
+    content = "A\n";
+  }
+
+  putFlowFile({}, content);
+
+  std::map<std::string, FlowFilePatternVec> expected{
+      {"matched", {content}},
+      {"unmatched", {}},
+      {"original", {content}}
+  };
+
+  run();
+
+  verifyAllOutput(expected);
+}
+
+TEST_CASE_METHOD(RouteTextController, "RouteText can group segments") {
+  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::GroupingFallbackValue, "GROUPING_FAILURE :(");
+
+  proc_->setDynamicProperty("A", "toA");
+  proc_->setDynamicProperty("B", "toB");
+
+  createOutput({"A", ""});
+  createOutput({"B", ""});
+
+  std::string content =
+      "group1.toA(one)\ngroup1.toB(two)\ngroup1.toA(three)\ngroup2.toA(four)\n"
+      "no_group.toA(five)\nno_group.toA(six)\ntoNone1\ngroup1.toNone2\ngroup1.toNone3";
+
+  putFlowFile({}, content);
+
+  std::map<std::string, FlowFilePatternVec> expected;
+
+  expected["A"] = {
+      FlowFilePattern{}.attr("RouteText.Group", "1").content("group1.toA(one)\ngroup1.toA(three)\n"),
+      FlowFilePattern{}.attr("RouteText.Group", "2").content("group2.toA(four)\n"),
+      FlowFilePattern{}.attr("RouteText.Group", "GROUPING_FAILURE :(").content("no_group.toA(five)\nno_group.toA(six)\n")
+  };
+  expected["B"] = {
+      FlowFilePattern{}.attr("RouteText.Group", "1").content("group1.toB(two)\n")
+  };
+  expected["matched"] = FlowFilePatternVec{};
+  expected["unmatched"] = {
+      FlowFilePattern{}.attr("RouteText.Group", "1").content("group1.toNone2\ngroup1.toNone3"),
+      FlowFilePattern{}.attr("RouteText.Group", "GROUPING_FAILURE :(").content("toNone1\n")
+  };
+  expected["original"] = {content};
+
+  run();
+
+  verifyAllOutput(expected);
+}
+
+TEST_CASE_METHOD(RouteTextController, "RouteText grouping uses empty strings for unused capture groups") {
+  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_->setDynamicProperty("A", "toA");
+
+  createOutput({"A", ""});
+
+  std::string content =
+      "group1.1:toA(one)\ngroup1.1:toA(two)\ngroup1.2:toA(three)\ngroup2:toA(four)\ngroup2:toA(five)";
+
+  putFlowFile({}, content);
+
+  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", "2, ").content("group2:toA(four)\ngroup2:toA(five)")
+  };
+  expected["matched"] = FlowFilePatternVec{};
+  expected["unmatched"] = FlowFilePatternVec{};
+  expected["original"] = {content};
+
+  run();
+
+  verifyAllOutput(expected);
+}
+
+TEST_CASE_METHOD(RouteTextController, "RouteText can match on Full Text") {
+  proc_->setProperty(processors::RouteText::SegmentationStrategy, "Full Text");
+  proc_->setProperty(processors::RouteText::MatchingStrategy, "Contains");
+
+  proc_->setDynamicProperty("A", "toA");
+
+  createOutput({"A", ""});
+
+  std::string content = "toA\r\ntoA\ntoA\r";
+
+  putFlowFile({}, content);
+
+  std::map<std::string, FlowFilePatternVec> expected{
+      {"matched", {}},
+      {"unmatched", {}},
+      {"A", {content}},
+      {"original", {content}}
+  };
+
+  run();
+
+  verifyAllOutput(expected);
+}
+
+TEST_CASE_METHOD(RouteTextController, "Expressions have access to injected variables") {
+  proc_->setProperty(processors::RouteText::SegmentationStrategy, "Per Line");
+  proc_->setProperty(processors::RouteText::MatchingStrategy, "Satisfies Expression");
+
+  SECTION("Segment") {
+    proc_->setDynamicProperty("A1", "${segment:startsWith('toA'):and(${segmentNo:equals('1')})}");
+    proc_->setDynamicProperty("A2", "${segment:startsWith('toA'):and(${segmentNo:equals('2')})}");
+  }
+  SECTION("Line") {
+    proc_->setDynamicProperty("A1", "${line:startsWith('toA'):and(${lineNo:equals('1')})}");
+    proc_->setDynamicProperty("A2", "${line:startsWith('toA'):and(${lineNo:equals('2')})}");
+  }
+
+  createOutput({"A1", ""});
+  createOutput({"A2", ""});
+
+  std::string content = "toA one\ntoA two";
+
+  putFlowFile({}, content);
+
+  std::map<std::string, FlowFilePatternVec> expected{
+      {"matched", {}},
+      {"unmatched", {}},
+      {"A1", {FlowFilePattern{}.attr("line", {}).attr("lineNo", {}).attr("segment", {}).attr("segmentNo", {}).content("toA one\n")}},
+      {"A2", {FlowFilePattern{}.attr("line", {}).attr("lineNo", {}).attr("segment", {}).attr("segmentNo", {}).content("toA two")}},
+      {"original", {content}}
+  };
+
+  run();
+
+  verifyAllOutput(expected);
+}
diff --git a/libminifi/include/controllers/SSLContextService.h b/libminifi/include/controllers/SSLContextService.h
index a9c8b2e..a1975b8 100644
--- a/libminifi/include/controllers/SSLContextService.h
+++ b/libminifi/include/controllers/SSLContextService.h
@@ -218,7 +218,7 @@ class SSLContextService : public core::controller::ControllerService {
 #endif
 
   static bool isFileTypeP12(const std::string& filename) {
-    return utils::StringUtils::endsWithIgnoreCase(filename, "p12");
+    return utils::StringUtils::endsWith(filename, "p12", false);
   }
 
  private:
diff --git a/libminifi/include/core/ProcessContext.h b/libminifi/include/core/ProcessContext.h
index 18389b2..da8e343 100644
--- a/libminifi/include/core/ProcessContext.h
+++ b/libminifi/include/core/ProcessContext.h
@@ -102,7 +102,7 @@ class ProcessContext : public controller::ControllerServiceLookup, public core::
 
   template<typename T = std::string>
   std::enable_if_t<std::is_default_constructible<T>::value, std::optional<T>>
-  getProperty(const Property& property) {
+  getProperty(const Property& property) const {
     T value;
     try {
       if (!getProperty(property.getName(), value)) return std::nullopt;
@@ -126,6 +126,23 @@ class ProcessContext : public controller::ControllerServiceLookup, public core::
   virtual bool getDynamicProperty(const Property &property, std::string &value, const std::shared_ptr<FlowFile>& /*flow_file*/) {
     return getDynamicProperty(property.getName(), value);
   }
+  bool getDynamicProperty(const Property &property, std::string &value, const std::shared_ptr<FlowFile>& flow_file, const std::map<std::string, std::string>& variables) {
+    std::map<std::string, std::optional<std::string>> original_attributes;
+    for (const auto& [variable, value] : variables) {
+      original_attributes[variable] = flow_file->getAttribute(variable);
+      flow_file->setAttribute(variable, value);
+    }
+    auto onExit = gsl::finally([&]{
+      for (const auto& attr : original_attributes) {
+        if (attr.second) {
+          flow_file->setAttribute(attr.first, attr.second.value());
+        } else {
+          flow_file->removeAttribute(attr.first);
+        }
+      }
+    });
+    return getDynamicProperty(property, value, flow_file);
+  }
   std::vector<std::string> getDynamicPropertyKeys() const {
     return processor_node_->getDynamicPropertyKeys();
   }
diff --git a/libminifi/include/io/BufferStream.h b/libminifi/include/io/BufferStream.h
index 3a8fc7d..6fb2463 100644
--- a/libminifi/include/io/BufferStream.h
+++ b/libminifi/include/io/BufferStream.h
@@ -78,7 +78,7 @@ class BufferStream : public BaseStream {
    * Returns the underlying buffer
    * @return vector's array
    **/
-  const uint8_t *getBuffer() const override {
+  const uint8_t* getBuffer() const override {
     return buffer_.data();
   }
 
diff --git a/libminifi/include/io/Stream.h b/libminifi/include/io/Stream.h
index db1acea..45e55dc 100644
--- a/libminifi/include/io/Stream.h
+++ b/libminifi/include/io/Stream.h
@@ -53,6 +53,7 @@ class Stream {
   virtual const uint8_t* getBuffer() const {
     throw std::runtime_error("Not a buffered stream");
   }
+
   virtual ~Stream() = default;
 };
 
diff --git a/libminifi/include/utils/ProcessorConfigUtils.h b/libminifi/include/utils/ProcessorConfigUtils.h
index a5282c8..acf7fd5 100644
--- a/libminifi/include/utils/ProcessorConfigUtils.h
+++ b/libminifi/include/utils/ProcessorConfigUtils.h
@@ -38,6 +38,19 @@ std::chrono::milliseconds parseTimePropertyMSOrThrow(core::ProcessContext* conte
 std::optional<uint64_t> getOptionalUintProperty(const core::ProcessContext& context, const std::string& property_name);
 std::string parsePropertyWithAllowableValuesOrThrow(const core::ProcessContext& context, const std::string& property_name, const std::set<std::string>& allowable_values);
 
+template<typename T>
+T parseEnumProperty(const core::ProcessContext& context, const core::Property& prop) {
+  std::string value;
+  if (!context.getProperty(prop.getName(), value)) {
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Property '" + prop.getName() + "' is missing");
+  }
+  T result = T::parse(value.c_str(), T{});
+  if (!result) {
+    throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Property '" + prop.getName() + "' has invalid value: '" + value + "'");
+  }
+  return result;
+}
+
 }  // namespace utils
 }  // namespace minifi
 }  // namespace nifi
diff --git a/libminifi/include/utils/StringUtils.h b/libminifi/include/utils/StringUtils.h
index ff8c04c..0df9541 100644
--- a/libminifi/include/utils/StringUtils.h
+++ b/libminifi/include/utils/StringUtils.h
@@ -106,6 +106,10 @@ class StringUtils {
     return s;
   }
 
+  static std::string_view trim(std::string_view sv);
+
+  static std::string_view trim(const char* str);
+
   /**
    * Compares strings by lower casing them.
    */
@@ -128,6 +132,16 @@ class StringUtils {
     return std::equal(right, right + right_len, left, [](unsigned char lc, unsigned char rc) { return std::tolower(lc) == std::tolower(rc); });
   }
 
+  static inline bool equals(const std::string_view& left, const std::string_view& right, bool case_sensitive = true) {
+    if (case_sensitive) {
+      return left == right;
+    }
+    if (left.length() != right.length()) {
+      return false;
+    }
+    return std::equal(left.begin(), left.end(), right.begin(), [](unsigned char lc, unsigned char rc) { return std::tolower(lc) == std::tolower(rc); });
+  }
+
   static std::vector<std::string> split(const std::string &str, const std::string &delimiter);
   static std::vector<std::string> splitRemovingEmpty(const std::string& str, const std::string& delimiter);
   static std::vector<std::string> splitAndTrim(const std::string &str, const std::string &delimiter);
@@ -147,22 +161,24 @@ class StringUtils {
 
   static std::string& replaceAll(std::string& source_string, const std::string &from_string, const std::string &to_string);
 
-  inline static bool endsWithIgnoreCase(const std::string &value, const std::string & endString) {
-    if (endString.size() > value.size())
+  inline static bool startsWith(const std::string_view& value, const std::string_view& start, bool case_sensitive = true) {
+    if (start.length() > value.length()) {
       return false;
-    return std::equal(endString.rbegin(), endString.rend(), value.rbegin(), [](unsigned char lc, unsigned char rc) {return tolower(lc) == tolower(rc);});
-  }
-
-  inline static bool startsWith(const std::string& value, const std::string& start_string) {
-    if (start_string.size() > value.size())
-      return false;
-    return std::equal(start_string.begin(), start_string.end(), value.begin());
+    }
+    if (case_sensitive) {
+      return std::equal(start.begin(), start.end(), value.begin());
+    }
+    return std::equal(start.begin(), start.end(), value.begin(), [](unsigned char lc, unsigned char rc) {return tolower(lc) == tolower(rc);});
   }
 
-  inline static bool endsWith(const std::string& value, const std::string& end_string) {
-    if (end_string.size() > value.size())
+  inline static bool endsWith(const std::string_view& value, const std::string_view& end, bool case_sensitive = true) {
+    if (end.length() > value.length()) {
       return false;
-    return std::equal(end_string.rbegin(), end_string.rend(), value.rbegin());
+    }
+    if (case_sensitive) {
+      return std::equal(end.rbegin(), end.rend(), value.rbegin());
+    }
+    return std::equal(end.rbegin(), end.rend(), value.rbegin(), [](unsigned char lc, unsigned char rc) {return tolower(lc) == tolower(rc);});
   }
 
   inline static std::string hex_ascii(const std::string& in) {
diff --git a/libminifi/src/utils/StringUtils.cpp b/libminifi/src/utils/StringUtils.cpp
index 1732886..54ae0db 100644
--- a/libminifi/src/utils/StringUtils.cpp
+++ b/libminifi/src/utils/StringUtils.cpp
@@ -52,6 +52,20 @@ std::string StringUtils::trim(const std::string& s) {
   return trimRight(trimLeft(s));
 }
 
+std::string_view StringUtils::trim(std::string_view sv) {
+  auto begin = std::find_if(sv.begin(), sv.end(), [](unsigned char c) -> bool { return !isspace(c); });
+  auto end = std::find_if(sv.rbegin(), std::reverse_iterator(begin), [](unsigned char c) -> bool { return !isspace(c); }).base();
+  // c++20 iterator constructor
+  // return std::string_view(begin, end);
+  // but for now
+  // on windows std::string_view::const_iterator is not a const char*
+  return sv.substr(std::distance(sv.begin(), begin), std::distance(begin, end));
+}
+
+std::string_view StringUtils::trim(const char* str) {
+  return trim(std::string_view(str));
+}
+
 template<typename Fun>
 std::vector<std::string> split_transformed(std::string str, const std::string& delimiter, Fun transformation) {
   std::vector<std::string> result;
@@ -87,7 +101,7 @@ std::vector<std::string> StringUtils::splitRemovingEmpty(const std::string& str,
 }
 
 std::vector<std::string> StringUtils::splitAndTrim(const std::string& str, const std::string& delimiter) {
-  return split_transformed(str, delimiter, trim);
+  return split_transformed(str, delimiter, static_cast<std::string(*)(const std::string&)>(trim));
 }
 
 std::vector<std::string> StringUtils::splitAndTrimRemovingEmpty(const std::string& str, const std::string& delimiter) {
diff --git a/libminifi/test/TestBase.h b/libminifi/test/TestBase.h
index 8dd09c6..bbee252 100644
--- a/libminifi/test/TestBase.h
+++ b/libminifi/test/TestBase.h
@@ -385,3 +385,22 @@ static bool extensionInitializer = [] {
   return true;
 }();
 #endif
+
+namespace Catch {
+template<typename T>
+struct StringMaker<std::optional<T>> {
+  static std::string convert(const std::optional<T>& val) {
+    if (val) {
+      return "std::optional(" + StringMaker<T>::convert(val.value()) + ")";
+    }
+    return "std::nullopt";
+  }
+};
+
+template<>
+struct StringMaker<std::nullopt_t> {
+  static std::string convert(const std::nullopt_t& /*val*/) {
+    return "std::nullopt";
+  }
+};
+}  // namespace Catch
diff --git a/libminifi/test/unit/ProcessorConfigUtilsTests.cpp b/libminifi/test/unit/ProcessorConfigUtilsTests.cpp
new file mode 100644
index 0000000..b385b60
--- /dev/null
+++ b/libminifi/test/unit/ProcessorConfigUtilsTests.cpp
@@ -0,0 +1,62 @@
+/**
+ * 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 "../TestBase.h"
+#include "core/Processor.h"
+#include "utils/ProcessorConfigUtils.h"
+#include "utils/Enum.h"
+
+namespace org::apache::nifi::minifi::core {
+namespace {
+
+class TestProcessor : public Processor {
+  using Processor::Processor;
+};
+
+#pragma GCC diagnostic push
+#pragma GCC diagnostic warning "-Wunused-function"
+
+SMART_ENUM(TestEnum,
+  (A, "A"),
+  (B, "B")
+)
+
+#pragma GCC diagnostic pop
+
+TEST_CASE("Parse enum property") {
+  auto prop = PropertyBuilder::createProperty("prop")
+      ->withAllowableValues(TestEnum::values())
+      ->build();
+  auto proc = std::make_shared<TestProcessor>("test-proc");
+  proc->setSupportedProperties({prop});
+  ProcessContext context(std::make_shared<ProcessorNode>(proc), nullptr, nullptr, nullptr, nullptr, nullptr);
+  SECTION("Valid") {
+    proc->setProperty(prop, "B");
+    TestEnum val = utils::parseEnumProperty<TestEnum>(context, prop);
+    REQUIRE(val == TestEnum::B);
+  }
+  SECTION("Invalid") {
+    proc->setProperty(prop, "C");
+    REQUIRE_THROWS(utils::parseEnumProperty<TestEnum>(context, prop));
+  }
+  SECTION("Missing") {
+    REQUIRE_THROWS(utils::parseEnumProperty<TestEnum>(context, prop));
+  }
+}
+
+}  // namespace
+}  // namespace org::apache::nifi::minifi::core
diff --git a/libminifi/test/unit/StringUtilsTests.cpp b/libminifi/test/unit/StringUtilsTests.cpp
index 85f6292..303c21f 100644
--- a/libminifi/test/unit/StringUtilsTests.cpp
+++ b/libminifi/test/unit/StringUtilsTests.cpp
@@ -162,6 +162,7 @@ TEST_CASE("TestStringUtils::testJoin", "[test string join]") {
 }
 
 TEST_CASE("TestStringUtils::trim", "[test trim]") {
+  REQUIRE("" == StringUtils::trim(""));
   REQUIRE("" == StringUtils::trim(" \n\t"));
   REQUIRE("foobar" == StringUtils::trim("foobar"));
   REQUIRE("foo bar" == StringUtils::trim("foo bar"));
@@ -193,9 +194,11 @@ TEST_CASE("TestStringUtils::trim", "[test trim]") {
   REQUIRE("foobar\n\t " == StringUtils::trimLeft(" \n\tfoobar\n\t "));
 }
 
-TEST_CASE("TestStringUtils::startsWith", "[test startsWith]") {
+TEST_CASE("TestStringUtils::startsWith - case sensitive", "[test startsWith]") {
   REQUIRE(StringUtils::startsWith("abcd", ""));
   REQUIRE(StringUtils::startsWith("abcd", "a"));
+  REQUIRE(!StringUtils::startsWith("Abcd", "a"));
+  REQUIRE(!StringUtils::startsWith("abcd", "A"));
   REQUIRE(StringUtils::startsWith("abcd", "abcd"));
   REQUIRE(StringUtils::startsWith("abcd", "abc"));
   REQUIRE(!StringUtils::startsWith("abcd", "abcde"));
@@ -206,9 +209,11 @@ TEST_CASE("TestStringUtils::startsWith", "[test startsWith]") {
   REQUIRE(!StringUtils::startsWith("abcd", "d"));
 }
 
-TEST_CASE("TestStringUtils::endsWith", "[test endsWith]") {
+TEST_CASE("TestStringUtils::endsWith - case sensitive", "[test endsWith]") {
   REQUIRE(StringUtils::endsWith("abcd", ""));
   REQUIRE(StringUtils::endsWith("abcd", "d"));
+  REQUIRE(!StringUtils::endsWith("abcD", "d"));
+  REQUIRE(!StringUtils::endsWith("abcd", "D"));
   REQUIRE(StringUtils::endsWith("abcd", "abcd"));
   REQUIRE(StringUtils::endsWith("abcd", "bcd"));
   REQUIRE(!StringUtils::endsWith("abcd", "1abcd"));
@@ -219,6 +224,36 @@ TEST_CASE("TestStringUtils::endsWith", "[test endsWith]") {
   REQUIRE(!StringUtils::endsWith("abcd", "a"));
 }
 
+TEST_CASE("TestStringUtils::startsWith - case insensitive", "[test startsWith case insensitive]") {
+  REQUIRE(StringUtils::startsWith("abcd", "", false));
+  REQUIRE(StringUtils::startsWith("abcd", "a", false));
+  REQUIRE(StringUtils::startsWith("Abcd", "a", false));
+  REQUIRE(StringUtils::startsWith("abcd", "A", false));
+  REQUIRE(StringUtils::startsWith("aBcd", "abCd", false));
+  REQUIRE(StringUtils::startsWith("abcd", "abc", false));
+  REQUIRE(!StringUtils::startsWith("abcd", "abcde", false));
+
+  REQUIRE(StringUtils::startsWith("", "", false));
+  REQUIRE(!StringUtils::startsWith("", "abcd", false));
+  REQUIRE(!StringUtils::startsWith("abcd", "b", false));
+  REQUIRE(!StringUtils::startsWith("abcd", "d", false));
+}
+
+TEST_CASE("TestStringUtils::endsWith - case insensitive", "[test endsWith case insensitive]") {
+  REQUIRE(StringUtils::endsWith("abcd", "", false));
+  REQUIRE(StringUtils::endsWith("abcd", "d", false));
+  REQUIRE(StringUtils::endsWith("abcd", "D", false));
+  REQUIRE(StringUtils::endsWith("abcD", "d", false));
+  REQUIRE(StringUtils::endsWith("abcd", "abcd", false));
+  REQUIRE(StringUtils::endsWith("aBcd", "bcD", false));
+  REQUIRE(!StringUtils::endsWith("abCd", "1aBcd", false));
+
+  REQUIRE(StringUtils::endsWith("", "", false));
+  REQUIRE(!StringUtils::endsWith("", "abcd", false));
+  REQUIRE(!StringUtils::endsWith("abcd", "c", false));
+  REQUIRE(!StringUtils::endsWith("abcd", "a", false));
+}
+
 TEST_CASE("TestStringUtils::toBool", "[test toBool]") {
   std::vector<std::pair<std::string, std::optional<bool>>> cases{
       {"", {}},