You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by ab...@apache.org on 2019/07/09 11:55:48 UTC

[nifi-minifi-cpp] branch master updated: MINIFICPP-828 - Create a common regex utility

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 6c028c5  MINIFICPP-828 - Create a common regex utility
6c028c5 is described below

commit 6c028c5441ac86222a717933c7076b4edd856b42
Author: Nghia Le <mi...@gmail.com>
AuthorDate: Mon Jul 8 17:03:19 2019 +0200

    MINIFICPP-828 - Create a common regex utility
    
    Signed-off-by: Arpad Boda <ab...@apache.org>
    
    This closes #606
---
 extensions/http-curl/client/HTTPClient.cpp         |  21 +--
 extensions/sftp/processors/ListSFTP.cpp            |  57 +------
 extensions/sftp/processors/ListSFTP.h              |  15 +-
 .../standard-processors/processors/ExtractText.cpp | 102 +++----------
 .../standard-processors/processors/GetFile.cpp     |  18 +--
 .../standard-processors/processors/TailFile.cpp    |  19 +--
 libminifi/include/Exception.h                      |   3 +-
 libminifi/include/utils/RegexUtils.h               |  84 +++++++++++
 libminifi/src/utils/RegexUtils.cpp                 | 165 +++++++++++++++++++++
 libminifi/test/unit/RegexUtilsTests.cpp            |  72 +++++++++
 10 files changed, 371 insertions(+), 185 deletions(-)

diff --git a/extensions/http-curl/client/HTTPClient.cpp b/extensions/http-curl/client/HTTPClient.cpp
index f2be475..b42ed56 100644
--- a/extensions/http-curl/client/HTTPClient.cpp
+++ b/extensions/http-curl/client/HTTPClient.cpp
@@ -16,6 +16,7 @@
  * limitations under the License.
  */
 #include "HTTPClient.h"
+#include "Exception.h"
 #include <memory>
 #include <climits>
 #include <map>
@@ -23,6 +24,7 @@
 #include <string>
 #include <algorithm>
 #include "utils/StringUtils.h"
+#include "utils/RegexUtils.h"
 
 namespace org {
 namespace apache {
@@ -335,21 +337,12 @@ void HTTPClient::set_request_method(const std::string method) {
 bool HTTPClient::matches(const std::string &value, const std::string &sregex) {
   if (sregex == ".*")
     return true;
-
-#ifdef WIN32
-  std::regex rgx(sregex);
-  return std::regex_match(value, rgx);
-#else
-  regex_t regex;
-  int ret = regcomp(&regex, sregex.c_str(), 0);
-  if (ret)
-    return false;
-  ret = regexec(&regex, value.c_str(), (size_t) 0, NULL, 0);
-  regfree(&regex);
-  if (ret)
+  try {
+    utils::Regex rgx(sregex);
+    return rgx.match(value);
+  } catch (const Exception &e) {
     return false;
-#endif
-  return true;
+  }
 }
 
 void HTTPClient::configure_secure_connection(CURL *http_session) {
diff --git a/extensions/sftp/processors/ListSFTP.cpp b/extensions/sftp/processors/ListSFTP.cpp
index e4b56be..c53a887 100644
--- a/extensions/sftp/processors/ListSFTP.cpp
+++ b/extensions/sftp/processors/ListSFTP.cpp
@@ -36,6 +36,7 @@
 #include "utils/ByteArrayCallback.h"
 #include "utils/TimeUtil.h"
 #include "utils/StringUtils.h"
+#include "utils/RegexUtils.h"
 #include "utils/ScopeGuard.h"
 #include "utils/file/FileUtils.h"
 #include "core/FlowFile.h"
@@ -184,14 +185,6 @@ ListSFTP::ListSFTP(std::string name, utils::Identifier uuid /*= utils::Identifie
 }
 
 ListSFTP::~ListSFTP() {
-#ifndef WIN32
-  if (file_filter_regex_set_) {
-    regfree(&compiled_file_filter_regex_);
-  }
-  if (path_filter_regex_set_) {
-    regfree(&compiled_path_filter_regex_);
-  }
-#endif
 }
 
 void ListSFTP::onSchedule(const std::shared_ptr<core::ProcessContext> &context, const std::shared_ptr<core::ProcessSessionFactory> &sessionFactory) {
@@ -214,50 +207,24 @@ void ListSFTP::onSchedule(const std::shared_ptr<core::ProcessContext> &context,
     utils::StringUtils::StringToBool(value, follow_symlink_);
   }
   if (context->getProperty(FileFilterRegex.getName(), file_filter_regex_)) {
-#ifndef WIN32
-    if (file_filter_regex_set_) {
-      regfree(&compiled_file_filter_regex_);
-    }
-    int ret = regcomp(&compiled_file_filter_regex_, file_filter_regex_.c_str(), 0);
-    if (ret != 0) {
-      logger_->log_error("Failed to compile File Filter Regex \"%s\"", file_filter_regex_.c_str());
-      file_filter_regex_set_ = false;
-    } else {
-      file_filter_regex_set_ = true;
-    }
-#else
     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 (std::regex_error&) {
+    } catch (const Exception &e) {
       logger_->log_error("Failed to compile File Filter Regex \"%s\"", file_filter_regex_.c_str());
       file_filter_regex_set_ = false;
     }
-#endif
   } else {
     file_filter_regex_set_ = false;
   }
   if (context->getProperty(PathFilterRegex.getName(), path_filter_regex_)) {
-#ifndef WIN32
-    if (path_filter_regex_set_) {
-      regfree(&compiled_path_filter_regex_);
-    }
-    int ret = regcomp(&compiled_path_filter_regex_, path_filter_regex_.c_str(), 0);
-    if (ret != 0) {
-      logger_->log_error("Failed to compile Path Filter Regex \"%s\"", path_filter_regex_.c_str());
-      file_filter_regex_set_ = false;
-    } else {
-      path_filter_regex_set_ = true;
-    }
-#else
     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 (std::regex_error&) {
+    } catch (const Exception &e) {
       logger_->log_error("Failed to compile Path Filter Regex \"%s\"", path_filter_regex_.c_str());
       path_filter_regex_set_ = false;
     }
-#endif
   } else {
     path_filter_regex_set_ = false;
   }
@@ -444,12 +411,7 @@ bool ListSFTP::filterFile(const std::string& parent_path, const std::string& fil
   /* File Filter Regex */
   if (file_filter_regex_set_) {
     bool match = false;
-#ifndef WIN32
-    int ret = regexec(&compiled_file_filter_regex_, filename.c_str(), static_cast<size_t>(0), nullptr, 0);
-    match = ret == 0;
-#else
-    match = std::regex_match(filename, compiled_file_filter_regex_);
-#endif
+    match = compiled_file_filter_regex_.match(filename);
     if (!match) {
       logger_->log_debug("Ignoring \"%s/%s\" because it did not match the File Filter Regex \"%s\"",
                          parent_path.c_str(),
@@ -471,12 +433,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;
-#ifndef WIN32
-    int ret = regexec(&compiled_path_filter_regex_, dir_path.c_str(), static_cast<size_t>(0), nullptr, 0);
-    match = ret == 0;
-#else
-    match = std::regex_match(dir_path, compiled_path_filter_regex_);
-#endif
+    match = compiled_path_filter_regex_.match(dir_path);
     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 4fe32e2..f1017f6 100644
--- a/extensions/sftp/processors/ListSFTP.h
+++ b/extensions/sftp/processors/ListSFTP.h
@@ -22,11 +22,6 @@
 #include <map>
 #include <chrono>
 #include <cstdint>
-#ifndef WIN32
-#include <regex.h>
-#else
-#include <regex>
-#endif
 
 #include "SFTPProcessorBase.h"
 #include "utils/ByteArrayCallback.h"
@@ -38,6 +33,7 @@
 #include "core/Resource.h"
 #include "core/logging/LoggerConfiguration.h"
 #include "utils/Id.h"
+#include "utils/RegexUtils.h"
 #include "../client/SFTPClient.h"
 
 namespace org {
@@ -114,13 +110,8 @@ class ListSFTP : public SFTPProcessorBase {
   std::string path_filter_regex_;
   bool file_filter_regex_set_;
   bool path_filter_regex_set_;
-#ifndef WIN32
-  regex_t compiled_file_filter_regex_;
-  regex_t compiled_path_filter_regex_;
-#else
-  std::regex compiled_file_filter_regex_;
-  std::regex compiled_path_filter_regex_;
-#endif
+  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/ExtractText.cpp b/extensions/standard-processors/processors/ExtractText.cpp
index 7d2bf89..b218221 100644
--- a/extensions/standard-processors/processors/ExtractText.cpp
+++ b/extensions/standard-processors/processors/ExtractText.cpp
@@ -31,11 +31,7 @@
 #include "core/ProcessSession.h"
 #include "core/FlowFile.h"
 
-#if !defined(_WIN32)
-#if __cplusplus <= 201103L
-#include <regex.h>
-#endif
-#endif
+#include "utils/RegexUtils.h"
 
 namespace org {
 namespace apache {
@@ -150,11 +146,11 @@ int64_t ExtractText::ReadCallback::process(std::shared_ptr<io::BaseStream> strea
   }
 
   if (regex_mode) {
-    std::regex_constants::syntax_option_type regex_mode = std::regex_constants::ECMAScript;
+    std::vector<utils::Regex::Mode> rgx_mode;
 
     bool insensitive;
     if (ctx_->getProperty(InsensitiveMatch.getName(), insensitive) && insensitive) {
-      regex_mode |= std::regex_constants::icase;
+      rgx_mode.push_back(utils::Regex::Mode::ICASE);
     }
 
     bool ignoregroupzero;
@@ -178,84 +174,32 @@ int64_t ExtractText::ReadCallback::process(std::shared_ptr<io::BaseStream> strea
 
       int matchcount = 0;
 
-#if (__cplusplus > 201103L) || defined(_WIN32)
-
-      std::regex rgx;
-
       try {
-        rgx = std::regex(value, regex_mode);
-      } catch(const std::regex_error& e) {
-        logger_->log_error("%s error encountered when trying to construct regular expression from property (key: %s) value: %s",
-            e.what(), k, value);
-        continue;
-      }
-
-      std::smatch matches;
-
-      while (std::regex_search(workStr, matches, rgx)) {
-        size_t i = ignoregroupzero ? 1 : 0;
-
-        for (; i < matches.size(); ++i, ++matchcount) {
-          std::string attributeValue = matches[i].str();
-          if (attributeValue.length() > maxCaptureSize) {
-            attributeValue = attributeValue.substr(0, maxCaptureSize);
-          }
-          if (matchcount == 0) {
-            regexAttributes[k] = attributeValue;
-          }
-          regexAttributes[k + '.' + std::to_string(matchcount)] = attributeValue;
-        }
-        if (!repeatingcapture) {
-          break;
-        }
-        workStr = matches.suffix();
-      }
-#else
-
-      size_t maxGroups = std::count(value.begin(), value.end(), '(') + 1;
-
-      regex_t regexCompiled;
-      std::vector<regmatch_t> groups;
-      groups.reserve(maxGroups);
-
-      if (regcomp(&regexCompiled, value.c_str(), REG_EXTENDED | (insensitive ? REG_ICASE : 0))) {
-        logger_->log_error("error encountered when trying to construct regular expression from property (key: %s) value: %s",
-                            k, value);
-        continue;
-      }
-
-      while (regexec(&regexCompiled, workStr.c_str(), groups.capacity(), groups.data(), 0) == 0) {
-        size_t g = 0;
-        size_t match_len = 0;
-        for (g = 0; g < maxGroups; g++) {
-          if (groups[g].rm_so == -1) {
-            break;  // No more groups
-          }
-
-          if (g == 0) {
-            match_len = groups[g].rm_eo;
-            if (ignoregroupzero) {
-              continue;
+        utils::Regex rgx(value, rgx_mode);
+        while (rgx.match(workStr)) {
+          const std::vector<std::string> &matches = rgx.getResult();
+          size_t i = ignoregroupzero ? 1 : 0;
+
+          for (; i < matches.size(); ++i, ++matchcount) {
+            std::string attributeValue = matches[i];
+            if (attributeValue.length() > maxCaptureSize) {
+              attributeValue = attributeValue.substr(0, maxCaptureSize);
             }
+            if (matchcount == 0) {
+              regexAttributes[k] = attributeValue;
+            }
+            regexAttributes[k + '.' + std::to_string(matchcount)] = attributeValue;
           }
-
-          std::string attributeValue(workStr.begin() + groups[g].rm_so, workStr.begin() + groups[g].rm_eo);
-          if (attributeValue.length() > maxCaptureSize) {
-            attributeValue = attributeValue.substr(0, maxCaptureSize);
-          }
-
-          if (matchcount == 0) {
-            regexAttributes[k] = attributeValue;
+          if (!repeatingcapture) {
+            break;
           }
-          regexAttributes[k + '.' + std::to_string(matchcount)] = attributeValue;
-          matchcount++;
+          workStr = rgx.getSuffix();
         }
-        if (!repeatingcapture || (match_len >= workStr.length())) {
-          break;
-        }
-        workStr = workStr.substr(match_len + 1);
+      } 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;
       }
-#endif
     }
 
     for (const auto& kv : regexAttributes) {
diff --git a/extensions/standard-processors/processors/GetFile.cpp b/extensions/standard-processors/processors/GetFile.cpp
index b50d6fc..dfaa00e 100644
--- a/extensions/standard-processors/processors/GetFile.cpp
+++ b/extensions/standard-processors/processors/GetFile.cpp
@@ -37,6 +37,7 @@
 #include "utils/StringUtils.h"
 #include "utils/file/FileUtils.h"
 #include "utils/TimeUtil.h"
+#include "utils/RegexUtils.h"
 #include "core/ProcessContext.h"
 #include "core/ProcessSession.h"
 #include "core/TypedValues.h"
@@ -251,21 +252,12 @@ bool GetFile::acceptFile(std::string fullName, std::string name, const GetFileRe
 
     if (request.keepSourceFile == false && access(fullName.c_str(), W_OK) != 0)
       return false;
-#ifndef WIN32
-    regex_t regex;
-    int ret = regcomp(&regex, request.fileFilter.c_str(), 0);
-    if (ret)
-      return false;
-    ret = regexec(&regex, name.c_str(), (size_t) 0, NULL, 0);
-    regfree(&regex);
-    if (ret)
-      return false;
-#else
-    std::regex regex(request.fileFilter);
-    if (!std::regex_match(name, regex)) {
+
+    utils::Regex rgx(request.fileFilter);
+    if (!rgx.match(name)) {
       return false;
     }
-#endif
+
     metrics_->input_bytes_ += statbuf.st_size;
     metrics_->accepted_files_++;
     return true;
diff --git a/extensions/standard-processors/processors/TailFile.cpp b/extensions/standard-processors/processors/TailFile.cpp
index 2fd0861..f22ceb5 100644
--- a/extensions/standard-processors/processors/TailFile.cpp
+++ b/extensions/standard-processors/processors/TailFile.cpp
@@ -40,6 +40,7 @@
 #include "utils/file/PathUtils.h"
 #include "utils/TimeUtil.h"
 #include "utils/StringUtils.h"
+#include "utils/RegexUtils.h"
 #ifdef HAVE_REGEX_CPP
 #include <regex>
 #else
@@ -152,22 +153,8 @@ void TailFile::onSchedule(const std::shared_ptr<core::ProcessContext> &context,
 }
 
 bool TailFile::acceptFile(const std::string &fileFilter, const std::string &file) {
-#ifndef HAVE_REGEX_CPP
-  regex_t regex;
-  int ret = regcomp(&regex, fileFilter.c_str(), 0);
-  if (ret)
-  return false;
-  ret = regexec(&regex, file.c_str(), (size_t) 0, NULL, 0);
-  regfree(&regex);
-  if (ret)
-  return false;
-#else
-  std::regex regex(fileFilter);
-  if (!std::regex_match(file, regex)) {
-    return false;
-  }
-  return true;
-#endif
+  utils::Regex rgx(fileFilter);
+  return rgx.match(file);
 }
 
 std::string TailFile::trimLeft(const std::string& s) {
diff --git a/libminifi/include/Exception.h b/libminifi/include/Exception.h
index b539c39..c90224c 100644
--- a/libminifi/include/Exception.h
+++ b/libminifi/include/Exception.h
@@ -40,12 +40,13 @@ enum ExceptionType {
   PROCESS_SCHEDULE_EXCEPTION,
   SITE2SITE_EXCEPTION,
   GENERAL_EXCEPTION,
+  REGEX_EXCEPTION,
   MAX_EXCEPTION
 };
 
 // Exception String 
 static const char *ExceptionStr[MAX_EXCEPTION] = { "File Operation", "Flow File Operation", "Processor Operation", "Process Session Operation", "Process Schedule Operation", "Site2Site Protocol",
-    "General Operation" };
+    "General Operation", "Regex Operation" };
 
 // Exception Type to String 
 inline const char *ExceptionTypeToString(ExceptionType type) {
diff --git a/libminifi/include/utils/RegexUtils.h b/libminifi/include/utils/RegexUtils.h
new file mode 100644
index 0000000..4f7e2c6
--- /dev/null
+++ b/libminifi/include/utils/RegexUtils.h
@@ -0,0 +1,84 @@
+/**
+ *
+ * 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.
+ */
+
+#ifndef LIBMINIFI_INCLUDE_IO_REGEXUTILS_H_
+#define LIBMINIFI_INCLUDE_IO_REGEXUTILS_H_
+
+#include <vector>
+#include <regex>
+
+#if (__cplusplus > 201103L) || defined(_WIN32)
+#define NO_MORE_REGFREEE
+#endif
+
+#ifndef NO_MORE_REGFREEE
+#include <regex.h>
+#endif
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+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 &) = delete;
+  Regex& operator=(const Regex &) = delete;
+  Regex(Regex&& other);
+  Regex& operator=(Regex&& other);
+  ~Regex();
+  bool match(const std::string &pattern);
+  const std::vector<std::string>& getResult() const;
+  const std::string& getSuffix() const;
+
+ private:
+  std::string pat_;
+  std::string suffix_;
+  std::string regexStr_;
+  std::vector<std::string> results_;
+  bool valid_;
+
+#ifdef NO_MORE_REGFREEE
+
+  std::regex compiledRegex_;
+  std::regex_constants::syntax_option_type regex_mode_;
+  std::smatch matches_;
+
+#else
+
+  regex_t compiledRegex_;
+  int regex_mode_;
+  std::vector<regmatch_t> matches_;
+
+#endif
+};
+
+} /* namespace utils */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
+
+#endif /* LIBMINIFI_INCLUDE_IO_REGEXUTILS_H_ */
diff --git a/libminifi/src/utils/RegexUtils.cpp b/libminifi/src/utils/RegexUtils.cpp
new file mode 100644
index 0000000..b569cef
--- /dev/null
+++ b/libminifi/src/utils/RegexUtils.cpp
@@ -0,0 +1,165 @@
+/**
+ *
+ * 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 "Exception.h"
+#include <iostream>
+#include <vector>
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+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)
+    : regexStr_(value),
+      valid_(false) {
+  if (regexStr_.empty())
+    return;
+
+  // 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;
+    }
+  }
+  // Compile
+#ifdef NO_MORE_REGFREEE
+  try {
+    compiledRegex_ = std::regex(regexStr_, regex_mode_);
+    valid_ = true;
+  } catch (const std::regex_error &e) {
+    throw Exception(REGEX_EXCEPTION, e.what());
+  }
+#else
+  int err_code = regcomp(&compiledRegex_, regexStr_.c_str(), regex_mode_);
+  if (err_code) {
+    const size_t sz = regerror(err_code, &compiledRegex_, nullptr, 0);
+    std::vector<char> msg(sz);
+    regerror(err_code, &compiledRegex_, msg.data(), msg.size());
+    throw Exception(REGEX_EXCEPTION, std::string(msg.begin(), msg.end()));
+  }
+  valid_ = true;
+  int maxGroups = std::count(regexStr_.begin(), regexStr_.end(), '(') + 1;
+  matches_.resize(maxGroups);
+#endif
+}
+
+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;
+  }
+
+  pat_ = std::move(other.pat_);
+  suffix_ = std::move(other.suffix_);
+  regexStr_ = std::move(other.regexStr_);
+  results_ = std::move(other.results_);
+#ifdef NO_MORE_REGFREEE
+  compiledRegex_ = std::move(other.compiledRegex_);
+  regex_mode_ = other.regex_mode_;
+  matches_ = std::move(other.matches_);
+#else
+  if (valid_)
+    regfree(&compiledRegex_);
+  compiledRegex_ = other.compiledRegex_;
+  regex_mode_ = other.regex_mode_;
+  matches_ = std::move(other.matches_);
+#endif
+  valid_ = other.valid_;
+  other.valid_ = false;
+  return *this;
+}
+
+Regex::~Regex() {
+#ifndef NO_MORE_REGFREEE
+  if (valid_)
+    regfree(&compiledRegex_);
+#endif
+}
+
+bool Regex::match(const std::string &pattern) {
+  if (!valid_) {
+    return false;
+  }
+  results_.clear();
+  pat_ = pattern;
+#ifdef NO_MORE_REGFREEE
+  if (std::regex_search(pattern, matches_, compiledRegex_)) {
+    for (const auto &m : matches_) {
+      results_.push_back(m.str());
+    }
+    suffix_ = matches_.suffix();
+    return true;
+  }
+  return false;
+#else
+  if (regexec(&compiledRegex_, pattern.c_str(), matches_.size(),
+              matches_.data(), 0) == 0) {
+    for (const auto &m : matches_) {
+      if (m.rm_so == -1) {
+        break;
+      }
+      std::string s(pattern.begin() + m.rm_so, pattern.begin() + m.rm_eo);
+      results_.push_back(s);
+    }
+    if ((size_t) matches_[0].rm_eo >= pattern.size()) {
+      suffix_ = "";
+    } else {
+      suffix_ = pattern.substr(matches_[0].rm_eo + 1);
+    }
+    return true;
+  }
+  return false;
+#endif
+}
+
+const std::vector<std::string>& Regex::getResult() const { return results_; }
+
+const std::string& Regex::getSuffix() const { return suffix_; }
+
+} /* namespace utils */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
diff --git a/libminifi/test/unit/RegexUtilsTests.cpp b/libminifi/test/unit/RegexUtilsTests.cpp
new file mode 100644
index 0000000..961621a
--- /dev/null
+++ b/libminifi/test/unit/RegexUtilsTests.cpp
@@ -0,0 +1,72 @@
+/**
+ *
+ * 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 "utils/RegexUtils.h"
+#include "Exception.h"
+#include <string>
+#include <vector>
+
+using org::apache::nifi::minifi::utils::Regex;
+using org::apache::nifi::minifi::Exception;
+
+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(r1.match(pat));
+    auto ret = r1.getResult();
+    std::vector<std::string> ans = {"Speed limit 130", "130"};
+    REQUIRE(ans == ret);
+    REQUIRE("| Speed limit 80" == r1.getSuffix());
+}
+
+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 r2(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(!r1.match(pat));
+  Regex r2(rgx2, mode);
+  REQUIRE(!r2.match(pat));
+  REQUIRE(!r2.match("LMN"));
+  Regex r3(rgx3);
+  REQUIRE(r3.match(pat));
+}
+
+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(!r1.match(pat));
+  std::vector<Regex::Mode> mode = {Regex::Mode::ICASE};
+  Regex r2(rgx1, mode);
+  REQUIRE(r2.match(pat));
+}