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/03 22:45:55 UTC

[nifi-minifi-cpp] 01/02: MINIFICPP-925 - Hex encoder and decoder

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

commit 77f3c98c8121be1090e49e0c2df8c29873cc63c0
Author: Daniel Bakai <ba...@gmail.com>
AuthorDate: Sun Jun 30 19:05:59 2019 +0200

    MINIFICPP-925 - Hex encoder and decoder
    
    Signed-off-by: Arpad Boda <ab...@apache.org>
---
 extensions/bootstrap/docs/generatec2docs.h         |  11 +-
 extensions/pcap/CapturePacket.cpp                  |   4 +-
 extensions/pcap/CapturePacket.h                    |   2 +-
 extensions/sftp/client/SFTPClient.cpp              |   9 +-
 .../standard-processors/processors/HashContent.h   |  16 +-
 libminifi/include/utils/StringUtils.h              | 281 +++++++++++----------
 libminifi/src/utils/StringUtils.cpp                | 162 ++++++++++++
 libminifi/test/pcap-tests/CMakeLists.txt           |   8 +-
 libminifi/test/pcap-tests/PcapTest.cpp             |  23 +-
 libminifi/test/unit/StringUtilsTests.cpp           |  61 +++++
 10 files changed, 399 insertions(+), 178 deletions(-)

diff --git a/extensions/bootstrap/docs/generatec2docs.h b/extensions/bootstrap/docs/generatec2docs.h
index bcb3a3e..637900c 100644
--- a/extensions/bootstrap/docs/generatec2docs.h
+++ b/extensions/bootstrap/docs/generatec2docs.h
@@ -24,15 +24,6 @@
 #include <iomanip>
 #include "utils/StringUtils.h"
 
-std::string toHex(const std::string& s) {
-  std::ostringstream ret;
-
-  for (std::string::size_type i = 0; i < s.length(); ++i)
-    ret << std::hex << std::setfill('0') << std::setw(2) << (int) s[i];
-
-  return ret.str();
-}
-
 int generateC2Docs(const std::string &inputfile, const std::string &output) {
   std::ifstream inf(inputfile);
   std::string input((std::istreambuf_iterator<char>(inf)), std::istreambuf_iterator<char>());
@@ -81,7 +72,7 @@ int generateC2Docs(const std::string &inputfile, const std::string &output) {
     auto description = input.substr(0, nextBlock);
 
     auto desc = org::apache::nifi::minifi::utils::StringUtils::trim(description);
-    outputFile << "     extensions.insert(std::make_pair(\"" << processor << "\",utils::StringUtils::hex_ascii(\"" << toHex(desc) << "\")));\n";
+    outputFile << "     extensions.insert(std::make_pair(\"" << processor << "\",utils::StringUtils::from_hex(\"" << org::apache::nifi::minifi::utils::StringUtils::to_hex(desc) << "\")));\n";
   }
 
   outputFile << "}\n    return extensions[feature]; \n"
diff --git a/extensions/pcap/CapturePacket.cpp b/extensions/pcap/CapturePacket.cpp
index 6a07e2e..1b5d2e7 100644
--- a/extensions/pcap/CapturePacket.cpp
+++ b/extensions/pcap/CapturePacket.cpp
@@ -55,7 +55,7 @@ std::shared_ptr<utils::IdGenerator> CapturePacket::id_generator_ = utils::IdGene
 core::Property CapturePacket::BaseDir(core::PropertyBuilder::createProperty("Base Directory")->withDescription("Scratch directory for PCAP files")->withDefaultValue<std::string>("/tmp/")->build());
 
 core::Property CapturePacket::BatchSize(core::PropertyBuilder::createProperty("Batch Size")->withDescription("The number of packets to combine within a given PCAP")->withDefaultValue<uint64_t>(50)->build());
-core::Property CapturePacket::NetworkController("Network Controller", "Regular expression of the network controller(s) to which we will attach", ".*");
+core::Property CapturePacket::NetworkControllers("Network Controllers", "Regular expression of the network controller(s) to which we will attach", ".*");
 core::Property CapturePacket::CaptureBluetooth(core::PropertyBuilder::createProperty("Capture Bluetooth")->withDescription("True indicates that we support bluetooth interfaces")->withDefaultValue<bool>(false)->build());
 
 const char *CapturePacket::ProcessorName = "CapturePacket";
@@ -116,7 +116,7 @@ void CapturePacket::initialize() {
   // Set the supported properties
   std::set<core::Property> properties;
   properties.insert(BatchSize);
-  properties.insert(NetworkController);
+  properties.insert(NetworkControllers);
   properties.insert(BaseDir);
   properties.insert(CaptureBluetooth);
   setSupportedProperties(properties);
diff --git a/extensions/pcap/CapturePacket.h b/extensions/pcap/CapturePacket.h
index 1ec145d..6dfd276 100644
--- a/extensions/pcap/CapturePacket.h
+++ b/extensions/pcap/CapturePacket.h
@@ -109,7 +109,7 @@ class CapturePacket : public core::Processor {
   // Processor Name
   static const char *ProcessorName;
   static core::Property BatchSize;
-  static core::Property NetworkController;
+  static core::Property NetworkControllers;
   static core::Property BaseDir;
   static core::Property CaptureBluetooth;
   // Supported Relationships
diff --git a/extensions/sftp/client/SFTPClient.cpp b/extensions/sftp/client/SFTPClient.cpp
index bc27d5a..d2a2716 100644
--- a/extensions/sftp/client/SFTPClient.cpp
+++ b/extensions/sftp/client/SFTPClient.cpp
@@ -381,14 +381,15 @@ bool SFTPClient::connect() {
     if (fingerprint == nullptr) {
       logger_->log_warn("Cannot get remote server fingerprint");
     } else {
-      std::stringstream fingerprint_hex;
+      auto fingerprint_hex = utils::StringUtils::to_hex(reinterpret_cast<const uint8_t*>(fingerprint), 20);
+      std::stringstream fingerprint_hex_colon;
       for (size_t i = 0; i < 20; i++) {
-        fingerprint_hex << std::setfill('0') << std::setw(2) << std::hex << static_cast<int>(static_cast<uint8_t>(fingerprint[i]));
+        fingerprint_hex_colon << fingerprint_hex.substr(i * 2, 2);
         if (i != 19) {
-          fingerprint_hex << ":";
+          fingerprint_hex_colon << ":";
         }
       }
-      logger_->log_debug("SHA1 host key fingerprint for %s is %s", hostname_.c_str(), fingerprint_hex.str().c_str());
+      logger_->log_debug("SHA1 host key fingerprint for %s is %s", hostname_.c_str(), fingerprint_hex_colon.str().c_str());
     }
   }
 
diff --git a/extensions/standard-processors/processors/HashContent.h b/extensions/standard-processors/processors/HashContent.h
index bcfdd34..80a0bdb 100644
--- a/extensions/standard-processors/processors/HashContent.h
+++ b/extensions/standard-processors/processors/HashContent.h
@@ -38,21 +38,13 @@
 #include "core/ProcessSession.h"
 #include "core/Resource.h"
 #include "io/BaseStream.h"
+#include "utils/StringUtils.h"
 
 using HashReturnType = std::pair<std::string, int64_t>;
 
 namespace {
 #define HASH_BUFFER_SIZE 16384
 
-  std::string digestToString(const unsigned char * const digest, size_t size) {
-    std::stringstream ss;
-    for(size_t i = 0; i < size; i++)
-    {
-      ss << std::uppercase << std::hex << std::setw(2) << std::setfill('0') << (int)digest[i];
-    }
-    return ss.str();
-  }
-
   HashReturnType MD5Hash(const std::shared_ptr<org::apache::nifi::minifi::io::BaseStream>& stream) {
     HashReturnType ret_val;
     ret_val.second = 0;
@@ -72,7 +64,7 @@ namespace {
     if (ret_val.second > 0) {
       unsigned char digest[MD5_DIGEST_LENGTH];
       MD5_Final(digest, &context);
-      ret_val.first = digestToString(digest, MD5_DIGEST_LENGTH);
+      ret_val.first = utils::StringUtils::to_hex(digest, MD5_DIGEST_LENGTH, true /*uppercase*/);
     }
     return ret_val;
   }
@@ -96,7 +88,7 @@ namespace {
     if (ret_val.second > 0) {
       unsigned char digest[SHA_DIGEST_LENGTH];
       SHA1_Final(digest, &context);
-      ret_val.first = digestToString(digest, SHA_DIGEST_LENGTH);
+      ret_val.first = utils::StringUtils::to_hex(digest, SHA_DIGEST_LENGTH, true /*uppercase*/);
     }
     return ret_val;
   }
@@ -120,7 +112,7 @@ namespace {
     if (ret_val.second > 0) {
       unsigned char digest[SHA256_DIGEST_LENGTH];
       SHA256_Final(digest, &context);
-      ret_val.first = digestToString(digest, SHA256_DIGEST_LENGTH);
+      ret_val.first = utils::StringUtils::to_hex(digest, SHA256_DIGEST_LENGTH, true /*uppercase*/);
     }
     return ret_val;
   }
diff --git a/libminifi/include/utils/StringUtils.h b/libminifi/include/utils/StringUtils.h
index 326a048..6ecbebd 100644
--- a/libminifi/include/utils/StringUtils.h
+++ b/libminifi/include/utils/StringUtils.h
@@ -82,12 +82,7 @@ class StringUtils {
    * @param input input string
    * @param output output string.
    */
-  static bool StringToBool(std::string input, bool &output) {
-
-    std::transform(input.begin(), input.end(), input.begin(), ::tolower);
-    std::istringstream(input) >> std::boolalpha >> output;
-    return output;
-  }
+  static bool StringToBool(std::string input, bool &output);
 
   // Trim String utils
 
@@ -96,9 +91,7 @@ class StringUtils {
    * @param s incoming string
    * @returns modified string
    */
-  static std::string trim(std::string s) {
-    return trimRight(trimLeft(s));
-  }
+  static std::string trim(std::string s);
 
   /**
    * Trims left most part of a string
@@ -132,25 +125,7 @@ class StringUtils {
     }
   }
 
-  static std::vector<std::string> split(const std::string &str, const std::string &delimiter) {
-    std::vector<std::string> result;
-    auto curr = str.begin();
-    auto end = str.end();
-    auto is_func = [delimiter](int s) {
-      return delimiter.at(0) == s;
-    };
-    while (curr != end) {
-      curr = std::find_if_not(curr, end, is_func);
-      if (curr == end) {
-        break;
-      }
-      auto next = std::find_if(curr, end, is_func);
-      result.push_back(std::string(curr, next));
-      curr = next;
-    }
-
-    return result;
-  }
+  static std::vector<std::string> split(const std::string &str, const std::string &delimiter);
 
   /**
    * Converts a string to a float
@@ -158,82 +133,11 @@ class StringUtils {
    * @param output output float
    * @param cp failure policy
    */
-  static bool StringToFloat(std::string input, float &output, FailurePolicy cp = RETURN) {
-    try {
-      output = std::stof(input);
-    } catch (const std::invalid_argument &ie) {
-      switch (cp) {
-        case RETURN:
-        case NOTHING:
-          return false;
-        case EXIT:
-          exit(1);
-        case EXCEPT:
-          throw ie;
-      }
-    } catch (const std::out_of_range &ofr) {
-      switch (cp) {
-        case RETURN:
-        case NOTHING:
-          return false;
-        case EXIT:
-          exit(1);
-        case EXCEPT:
-          throw ofr;
-
-      }
-    }
+  static bool StringToFloat(std::string input, float &output, FailurePolicy cp = RETURN);
 
-    return true;
-
-  }
-
-  static std::string replaceEnvironmentVariables(std::string& original_string) {
-    int32_t beg_seq = 0;
-    int32_t end_seq = 0;
-    std::string source_string = original_string;
-    do {
-      beg_seq = source_string.find("${", beg_seq);
-      if (beg_seq > 0 && source_string.at(beg_seq - 1) == '\\') {
-        beg_seq += 2;
-        continue;
-      }
-      if (beg_seq < 0)
-        break;
-      end_seq = source_string.find("}", beg_seq + 2);
-      if (end_seq < 0)
-        break;
-      if (end_seq - (beg_seq + 2) < 0) {
-        beg_seq += 2;
-        continue;
-      }
-      const std::string env_field = source_string.substr(beg_seq + 2, end_seq - (beg_seq + 2));
-      const std::string env_field_wrapped = source_string.substr(beg_seq, end_seq + 1);
-      if (env_field.empty()) {
-        continue;
-      }
-      const auto strVal = std::getenv(env_field.c_str());
-      std::string env_value;
-      if (strVal != nullptr)
-        env_value = strVal;
-      source_string = replaceAll(source_string, env_field_wrapped, env_value);
-      beg_seq = 0;  // restart
-    } while (beg_seq >= 0);
+  static std::string replaceEnvironmentVariables(std::string& original_string);
 
-    source_string = replaceAll(source_string, "\\$", "$");
-
-    return source_string;
-  }
-
-  static std::string& replaceAll(std::string& source_string, const std::string &from_string, const std::string &to_string) {
-    std::size_t loc = 0;
-    std::size_t lastFound;
-    while ((lastFound = source_string.find(from_string, loc)) != std::string::npos) {
-      source_string.replace(lastFound, from_string.size(), to_string);
-      loc = lastFound + to_string.size();
-    }
-    return source_string;
-  }
+  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())
@@ -257,39 +161,15 @@ class StringUtils {
     }
     return newString;
   }
-
-  static std::string replaceMap(std::string source_string, const std::map<std::string, std::string> &replace_map) {
-    auto result_string = source_string;
-
-    std::vector<std::pair<size_t, std::pair<size_t, std::string>>> replacements;
-    for (const auto &replace_pair : replace_map) {
-      size_t replace_pos = 0;
-      while ((replace_pos = source_string.find(replace_pair.first, replace_pos)) != std::string::npos) {
-        replacements.emplace_back(std::make_pair(replace_pos, std::make_pair(replace_pair.first.length(), replace_pair.second)));
-        replace_pos += replace_pair.first.length();
-      }
-    }
-
-    std::sort(replacements.begin(), replacements.end(), [](const std::pair<size_t, std::pair<size_t, std::string>> a,
-        const std::pair<size_t, std::pair<size_t, std::string>> &b) {
-      return a.first > b.first;
-    });
-
-    for (const auto &replacement : replacements) {
-      result_string = source_string.replace(replacement.first, replacement.second.first, replacement.second.second);
-    }
-
-    return result_string;
-  }
-
+  
   /**
- * Concatenates strings stored in an arbitrary container using the provided separator.
- * @tparam TChar char type of the string (char or wchar_t)
- * @tparam U arbitrary container which has string or wstring value type
- * @param separator that is inserted between each elements. Type should match the type of strings in container.
- * @param container that contains the strings to be concatenated
- * @return the result string
- */
+   * Concatenates strings stored in an arbitrary container using the provided separator.
+   * @tparam TChar char type of the string (char or wchar_t)
+   * @tparam U arbitrary container which has string or wstring value type
+   * @param separator that is inserted between each elements. Type should match the type of strings in container.
+   * @param container that contains the strings to be concatenated
+   * @return the result string
+   */
   template<class TChar, class U, typename std::enable_if<std::is_same<typename U::value_type, std::basic_string<TChar>>::value>::type* = nullptr>
   static std::basic_string<TChar> join(const std::basic_string<TChar>& separator, const U& container) {
     typedef typename U::const_iterator ITtype;
@@ -347,6 +227,139 @@ class StringUtils {
     return join(std::basic_string<TChar>(separator), container);
   };
 
+  /**
+   * Hexdecodes the hexencoded string in data, ignoring every character that is not [0-9a-fA-F]
+   * @param data the output buffer where the hexdecoded bytes will be written. Must be at least length / 2 bytes long.
+   * @param data_length pointer to the length of data the data buffer. It will be filled with the length of the decoded bytes.
+   * @param hex the hexencoded string
+   * @param hex_length the length of hex
+   * @return true on success
+   */
+  inline static bool from_hex(uint8_t* data, size_t* data_length, const char* hex, size_t hex_length) {
+    if (*data_length < hex_length / 2) {
+      return false;
+    }
+    uint8_t n1;
+    bool found_first_nibble = false;
+    *data_length = 0;
+    for (size_t i = 0; i < hex_length; i++) {
+      const uint8_t byte = static_cast<uint8_t>(hex[i]);
+      if (byte > 127) {
+        continue;
+      }
+      uint8_t n = hex_lut[byte];
+      if (n != SKIP) {
+        if (found_first_nibble) {
+          data[(*data_length)++] = n1 << 4 | n;
+          found_first_nibble = false;
+        } else {
+          n1 = n;
+          found_first_nibble = true;
+        }
+      }
+    }
+    if (found_first_nibble) {
+      return false;
+    }
+    return true;
+  }
+
+  /**
+   * Hexdecodes a string
+   * @param hex the hexencoded string
+   * @param hex_length the length of hex
+   * @return the vector containing the hexdecoded bytes
+   */
+  inline static std::vector<uint8_t> from_hex(const char* hex, size_t hex_length) {
+    std::vector<uint8_t> decoded(hex_length / 2);
+    size_t data_length = decoded.size();
+    if (!from_hex(decoded.data(), &data_length, hex, hex_length)) {
+      throw std::runtime_error("Hexencoded string is malformatted");
+    }
+    decoded.resize(data_length);
+    return decoded;
+  }
+
+  /**
+   * Hexdecodes a string
+   * @param hex the hexencoded string
+   * @return the hexdecoded string
+   */
+  inline static std::string from_hex(const std::string& hex) {
+    auto data = from_hex(hex.data(), hex.length());
+    return std::string(reinterpret_cast<char*>(data.data()), data.size());
+  }
+
+  /**
+   * Hexencodes bytes and writes the result to hex
+   * @param hex the output buffer where the hexencoded string will be written. Must be at least length * 2 bytes long.
+   * @param data the bytes to be hexencoded
+   * @param length the length of data. Must not be larger than std::numeric_limits<size_t>::max()
+   * @param uppercase whether the hexencoded string should be upper case
+   */
+  inline static void to_hex(char* hex, const uint8_t* data, size_t length, bool uppercase) {
+    for (size_t i = 0; i < length; i++) {
+      hex[i * 2] = nibble_to_hex(data[i] >> 4, uppercase);
+      hex[i * 2 + 1] = nibble_to_hex(data[i] & 0xf, uppercase);
+    }
+  }
+
+  /**
+   * Creates a hexencoded string from data
+   * @param data the bytes to be hexencoded
+   * @param length the length of the data
+   * @param uppercase whether the hexencoded string should be upper case
+   * @return the hexencoded string
+   */
+  inline static std::string to_hex(const uint8_t* data, size_t length, bool uppercase = false) {
+    if (length > (std::numeric_limits<size_t>::max)() / 2) {
+      throw std::length_error("Data is too large to be hexencoded");
+    }
+    std::vector<char> buf(length * 2);
+    to_hex(buf.data(), data, length, uppercase);
+    return std::string(buf.data(), buf.size());
+  }
+
+  /**
+   * Hexencodes a string
+   * @param str the string to be hexencoded
+   * @param uppercase whether the hexencoded string should be upper case
+   * @return the hexencoded string
+   */
+  inline static std::string to_hex(const std::string& str, bool uppercase = false) {
+    return to_hex(reinterpret_cast<const uint8_t*>(str.data()), str.length(), uppercase);
+  }
+
+  static std::string replaceMap(std::string source_string, const std::map<std::string, std::string> &replace_map);
+
+ private:
+  inline static char nibble_to_hex(uint8_t nibble, bool uppercase) {
+    if (nibble < 10) {
+      return '0' + nibble;
+    } else {
+      return (uppercase ? 'A' : 'a') + nibble - 10;
+    }
+  }
+
+  static constexpr uint8_t SKIP = 255;
+  static constexpr uint8_t hex_lut[128] =
+      {SKIP, SKIP, SKIP, SKIP, SKIP, SKIP, SKIP, SKIP,
+       SKIP, SKIP, SKIP, SKIP, SKIP, SKIP, SKIP, SKIP,
+       SKIP, SKIP, SKIP, SKIP, SKIP, SKIP, SKIP, SKIP,
+       SKIP, SKIP, SKIP, SKIP, SKIP, SKIP, SKIP, SKIP,
+       SKIP, SKIP, SKIP, SKIP, SKIP, SKIP, SKIP, SKIP,
+       SKIP, SKIP, SKIP, SKIP, SKIP, SKIP, SKIP, SKIP,
+       0x00, 0x01, 0x02, 0x03, 0x04, 0x05, 0x06, 0x07,
+       0x08, 0x09, SKIP, SKIP, SKIP, SKIP, SKIP, SKIP,
+       SKIP, 0x0a, 0x0b, 0x0c, 0x0d, 0x0e, 0x0f, SKIP,
+       SKIP, SKIP, SKIP, SKIP, SKIP, SKIP, SKIP, SKIP,
+       SKIP, SKIP, SKIP, SKIP, SKIP, SKIP, SKIP, SKIP,
+       SKIP, SKIP, SKIP, SKIP, SKIP, SKIP, SKIP, SKIP,
+       SKIP, 0x0a, 0x0b, 0x0c, 0x0d, 0x0e, 0x0f, SKIP,
+       SKIP, SKIP, SKIP, SKIP, SKIP, SKIP, SKIP, SKIP,
+       SKIP, SKIP, SKIP, SKIP, SKIP, SKIP, SKIP, SKIP,
+       SKIP, SKIP, SKIP, SKIP, SKIP, SKIP, SKIP, SKIP};
+
 };
 
 } /* namespace utils */
diff --git a/libminifi/src/utils/StringUtils.cpp b/libminifi/src/utils/StringUtils.cpp
new file mode 100644
index 0000000..788327b
--- /dev/null
+++ b/libminifi/src/utils/StringUtils.cpp
@@ -0,0 +1,162 @@
+/**
+ * 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/StringUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+bool StringUtils::StringToBool(std::string input, bool &output) {
+  std::transform(input.begin(), input.end(), input.begin(), ::tolower);
+  std::istringstream(input) >> std::boolalpha >> output;
+  return output;
+}
+
+std::string StringUtils::trim(std::string s) {
+  return trimRight(trimLeft(s));
+}
+
+std::vector<std::string> StringUtils::split(const std::string &str, const std::string &delimiter) {
+  std::vector<std::string> result;
+  auto curr = str.begin();
+  auto end = str.end();
+  auto is_func = [delimiter](int s) {
+    return delimiter.at(0) == s;
+  };
+  while (curr != end) {
+    curr = std::find_if_not(curr, end, is_func);
+    if (curr == end) {
+      break;
+    }
+    auto next = std::find_if(curr, end, is_func);
+    result.push_back(std::string(curr, next));
+    curr = next;
+  }
+
+  return result;
+}
+
+bool StringUtils::StringToFloat(std::string input, float &output, FailurePolicy cp /*= RETURN*/) {
+  try {
+    output = std::stof(input);
+  } catch (const std::invalid_argument &ie) {
+    switch (cp) {
+      case RETURN:
+      case NOTHING:
+        return false;
+      case EXIT:
+        exit(1);
+      case EXCEPT:
+        throw ie;
+    }
+  } catch (const std::out_of_range &ofr) {
+    switch (cp) {
+      case RETURN:
+      case NOTHING:
+        return false;
+      case EXIT:
+        exit(1);
+      case EXCEPT:
+        throw ofr;
+    }
+  }
+
+  return true;
+}
+
+std::string StringUtils::replaceEnvironmentVariables(std::string& original_string) {
+  int32_t beg_seq = 0;
+  int32_t end_seq = 0;
+  std::string source_string = original_string;
+  do {
+    beg_seq = source_string.find("${", beg_seq);
+    if (beg_seq > 0 && source_string.at(beg_seq - 1) == '\\') {
+      beg_seq += 2;
+      continue;
+    }
+    if (beg_seq < 0)
+      break;
+    end_seq = source_string.find("}", beg_seq + 2);
+    if (end_seq < 0)
+      break;
+    if (end_seq - (beg_seq + 2) < 0) {
+      beg_seq += 2;
+      continue;
+    }
+    const std::string env_field = source_string.substr(beg_seq + 2, end_seq - (beg_seq + 2));
+    const std::string env_field_wrapped = source_string.substr(beg_seq, end_seq + 1);
+    if (env_field.empty()) {
+      continue;
+    }
+    const auto strVal = std::getenv(env_field.c_str());
+    std::string env_value;
+    if (strVal != nullptr)
+      env_value = strVal;
+    source_string = replaceAll(source_string, env_field_wrapped, env_value);
+    beg_seq = 0;  // restart
+  } while (beg_seq >= 0);
+
+  source_string = replaceAll(source_string, "\\$", "$");
+
+  return source_string;
+}
+
+std::string& StringUtils::replaceAll(std::string& source_string, const std::string &from_string, const std::string &to_string) {
+  std::size_t loc = 0;
+  std::size_t lastFound;
+  while ((lastFound = source_string.find(from_string, loc)) != std::string::npos) {
+    source_string.replace(lastFound, from_string.size(), to_string);
+    loc = lastFound + to_string.size();
+  }
+  return source_string;
+}
+
+std::string StringUtils::replaceMap(std::string source_string, const std::map<std::string, std::string> &replace_map) {
+  auto result_string = source_string;
+
+  std::vector<std::pair<size_t, std::pair<size_t, std::string>>> replacements;
+  for (const auto &replace_pair : replace_map) {
+    size_t replace_pos = 0;
+    while ((replace_pos = source_string.find(replace_pair.first, replace_pos)) != std::string::npos) {
+      replacements.emplace_back(std::make_pair(replace_pos, std::make_pair(replace_pair.first.length(), replace_pair.second)));
+      replace_pos += replace_pair.first.length();
+    }
+  }
+
+  std::sort(replacements.begin(), replacements.end(), [](const std::pair<size_t, std::pair<size_t, std::string>> a,
+                                                         const std::pair<size_t, std::pair<size_t, std::string>> &b) {
+    return a.first > b.first;
+  });
+
+  for (const auto &replacement : replacements) {
+    result_string = source_string.replace(replacement.first, replacement.second.first, replacement.second.second);
+  }
+
+  return result_string;
+}
+
+constexpr uint8_t StringUtils::SKIP;
+constexpr uint8_t StringUtils::hex_lut[128];
+
+} /* namespace utils */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
diff --git a/libminifi/test/pcap-tests/CMakeLists.txt b/libminifi/test/pcap-tests/CMakeLists.txt
index 14b4528..f66333c 100644
--- a/libminifi/test/pcap-tests/CMakeLists.txt
+++ b/libminifi/test/pcap-tests/CMakeLists.txt
@@ -29,17 +29,17 @@ FOREACH(testfile ${PCAP_TESTS})
   	target_include_directories(${testfilename} PRIVATE BEFORE "${CMAKE_BINARY_DIR}/extensions/pcap/pcap++/Dist/header/")
     createTests("${testfilename}")
     if(APPLE)    
-    	target_link_libraries ("${testfilename}" -Wl,-all_load minifi-pcap )
+        target_link_libraries ("${testfilename}" -Wl,-all_load minifi-pcap minifi-standard-processors)
 	else()
-  		target_link_libraries ("${testfilename}" -Wl,--whole-archive minifi-pcap -Wl,--no-whole-archive)
+        target_link_libraries ("${testfilename}" -Wl,--whole-archive minifi-pcap minifi-standard-processors -Wl,--no-whole-archive)
   	endif()
   MATH(EXPR PCAP_INT_TEST_COUNT "${PCAP_INT_TEST_COUNT}+1")
 ENDFOREACH()
 
 message("-- Finished building ${PCAP_INT_TEST_COUNT} libPCAP test file(s)...")
 if(APPLE)    
-	add_test(NAME PcapTest COMMAND PcapTest "${TEST_RESOURCES}/TestPcap.yml"  "${TEST_RESOURCES}/")
+    add_test(NAME PcapTest COMMAND PcapTest "${TEST_RESOURCES}/TestPcap.yml"  "${TEST_RESOURCES}/")
 else()
-	add_test(NAME PcapTest COMMAND PcapTest "${TEST_RESOURCES}/TestPcapLinux.yml"  "${TEST_RESOURCES}/")
+    add_test(NAME PcapTest COMMAND PcapTest "${TEST_RESOURCES}/TestPcapLinux.yml"  "${TEST_RESOURCES}/")
 endif()
 
diff --git a/libminifi/test/pcap-tests/PcapTest.cpp b/libminifi/test/pcap-tests/PcapTest.cpp
index 3c6fd27..eea0364 100644
--- a/libminifi/test/pcap-tests/PcapTest.cpp
+++ b/libminifi/test/pcap-tests/PcapTest.cpp
@@ -40,6 +40,7 @@
 #include "../unit/ProvenanceTestHelper.h"
 #include "io/StreamFactory.h"
 #include "core/ConfigurableComponent.h"
+#include "core/state/ProcessorController.h"
 #include "../integration/IntegrationBase.h"
 #include "CapturePacket.h"
 
@@ -71,20 +72,20 @@ class PcapTestHarness : public IntegrationBase {
     assert(LogTestController::getInstance().contains("Accepting ") == true && LogTestController::getInstance().contains("because it matches .*") );
   }
 
-  void queryRootProcessGroup(std::shared_ptr<core::ProcessGroup> pg) {
-    std::shared_ptr<core::Processor> proc = pg->findProcessor("pcap");
-    assert(proc != nullptr);
-
-    auto inv = std::dynamic_pointer_cast<minifi::processors::CapturePacket>(proc);
-    assert(inv != nullptr);
-
-    configuration->set(minifi::processors::CapturePacket::BaseDir.getName(), dir);
-    configuration->set(minifi::processors::CapturePacket::NetworkController.getName(), ".*");
-    configuration->set("nifi.c2.enable", "false");
+  void updateProperties(std::shared_ptr<minifi::FlowController> fc) {
+    auto components = fc->getComponents("pcap");
+    for (const auto& component : components) {
+      auto proccontroller = std::dynamic_pointer_cast<minifi::state::ProcessorController>(component);
+      if (proccontroller) {
+        auto processor = proccontroller->getProcessor();
+        processor->setProperty(minifi::processors::CapturePacket::BaseDir.getName(), dir);
+        processor->setProperty(minifi::processors::CapturePacket::NetworkControllers.getName(), ".*");
+      }
+    }
   }
 
  protected:
-  char *dir;
+  std::string dir;
   TestController testController;
 };
 
diff --git a/libminifi/test/unit/StringUtilsTests.cpp b/libminifi/test/unit/StringUtilsTests.cpp
index 6486f29..16704b4 100644
--- a/libminifi/test/unit/StringUtilsTests.cpp
+++ b/libminifi/test/unit/StringUtilsTests.cpp
@@ -20,6 +20,9 @@
 #include <list>
 #include <vector>
 #include <cstdlib>
+#include <random>
+#include <algorithm>
+#include <cstdint>
 #include "../TestBase.h"
 #include "core/Core.h"
 #include "utils/StringUtils.h"
@@ -112,3 +115,61 @@ TEST_CASE("TestStringUtils::testJoin", "[test string join]") {
 
   REQUIRE(StringUtils::join("this separator wont appear", std::vector<std::string>()) == "");
 }
+
+TEST_CASE("TestStringUtils::testHexEncode", "[test hex encode]") {
+  REQUIRE("" == StringUtils::to_hex(""));
+  REQUIRE("6f" == StringUtils::to_hex("o"));
+  REQUIRE("666f6f626172" == StringUtils::to_hex("foobar"));
+  REQUIRE("000102030405060708090a0b0c0d0e0f" == StringUtils::to_hex({0x00, 0x01, 0x02, 0x03,
+                                                                     0x04, 0x05, 0x06, 0x07,
+                                                                     0x08, 0x09, 0x0a, 0x0b,
+                                                                     0x0c, 0x0d, 0x0e, 0x0f}));
+  REQUIRE("6F" == StringUtils::to_hex("o", true /*uppercase*/));
+  REQUIRE("666F6F626172" == StringUtils::to_hex("foobar", true /*uppercase*/));
+  REQUIRE("000102030405060708090A0B0C0D0E0F" == StringUtils::to_hex({0x00, 0x01, 0x02, 0x03,
+                                                                     0x04, 0x05, 0x06, 0x07,
+                                                                     0x08, 0x09, 0x0a, 0x0b,
+                                                                     0x0c, 0x0d, 0x0e, 0x0f}, true /*uppercase*/));
+}
+
+TEST_CASE("TestStringUtils::testHexDecode", "[test hex decode]") {
+  REQUIRE("" == StringUtils::from_hex(""));
+  REQUIRE("o" == StringUtils::from_hex("6f"));
+  REQUIRE("o" == StringUtils::from_hex("6F"));
+  REQUIRE("foobar" == StringUtils::from_hex("666f6f626172"));
+  REQUIRE("foobar" == StringUtils::from_hex("666F6F626172"));
+  REQUIRE("foobar" == StringUtils::from_hex("66:6F:6F:62:61:72"));
+  REQUIRE("foobar" == StringUtils::from_hex("66 6F 6F 62 61 72"));
+  REQUIRE(std::string({0x00, 0x01, 0x02, 0x03,
+                       0x04, 0x05, 0x06, 0x07,
+                       0x08, 0x09, 0x0a, 0x0b,
+                       0x0c, 0x0d, 0x0e, 0x0f}) == StringUtils::from_hex("000102030405060708090a0b0c0d0e0f"));
+  REQUIRE(std::string({0x00, 0x01, 0x02, 0x03,
+                       0x04, 0x05, 0x06, 0x07,
+                       0x08, 0x09, 0x0a, 0x0b,
+                       0x0c, 0x0d, 0x0e, 0x0f}) == StringUtils::from_hex("000102030405060708090A0B0C0D0E0F"));
+  try {
+    StringUtils::from_hex("666f6f62617");
+    abort();
+  } catch (std::exception& e) {
+    REQUIRE(std::string("Hexencoded string is malformatted") == e.what());
+  }
+  try {
+    StringUtils::from_hex("666f6f6261 7");
+    abort();
+  } catch (std::exception& e) {
+    REQUIRE(std::string("Hexencoded string is malformatted") == e.what());
+  }
+}
+
+TEST_CASE("TestStringUtils::testHexEncodeDecode", "[test hex encode decode]") {
+  std::mt19937 gen(std::random_device { }());
+  const bool uppercase = gen() % 2;
+  const size_t length = gen() % 1024;
+  std::vector<uint8_t> data(length);
+  std::generate_n(data.begin(), data.size(), [&]() -> uint8_t {
+    return gen() % 256;
+  });
+  auto hex = utils::StringUtils::to_hex(data.data(), data.size(), uppercase);
+  REQUIRE(data == utils::StringUtils::from_hex(hex.data(), hex.size()));
+}