You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by GitBox <gi...@apache.org> on 2019/01/09 22:53:58 UTC

[GitHub] asfgit closed pull request #460: MINIFICPP-479: Add processor property descriptor updates with c2 validation

asfgit closed pull request #460: MINIFICPP-479: Add processor property descriptor updates with c2 validation
URL: https://github.com/apache/nifi-minifi-cpp/pull/460
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/CMakeLists.txt b/CMakeLists.txt
index 6859b3f4..a5009924 100644
--- a/CMakeLists.txt
+++ b/CMakeLists.txt
@@ -199,7 +199,11 @@ if(USE_SYSTEM_ZLIB STREQUAL "OFF")
   message("Using bundled zlib")
 
   if (WIN32)
+  IF(CMAKE_BUILD_TYPE MATCHES RELEASE)
+  set(BYPRODUCT "thirdparty/zlib-install/lib/zlibstatic.lib")
+  else()
   set(BYPRODUCT "thirdparty/zlib-install/lib/zlibstaticd.lib")
+  endif()
   else()
   set(BYPRODUCT "thirdparty/zlib-install/lib/libz.a")
   endif()
@@ -257,7 +261,11 @@ if(NOT DISABLE_CURL AND (NOT USE_SYSTEM_CURL))
   set(CURL_CXX_FLAGS "${CURL_C_FLAGS}")
 
   if (WIN32)
-  set(BYPRODUCT "thirdparty/curl-install/lib/libcurl-d.lib")
+  IF(CMAKE_BUILD_TYPE MATCHES RELEASE)
+	set(BYPRODUCT "thirdparty/curl-install/lib/libcurl.lib")
+  else()
+  	set(BYPRODUCT "thirdparty/curl-install/lib/libcurl-d.lib")
+  endif()
   else()
   set(BYPRODUCT "thirdparty/curl-install/lib/libcurl.a")
   endif()
@@ -532,7 +540,15 @@ else()
 endif()
 # Generate source assembly
 set(ASSEMBLY_BASE_NAME "${CMAKE_PROJECT_NAME}-${PROJECT_VERSION_MAJOR}.${PROJECT_VERSION_MINOR}.${PROJECT_VERSION_PATCH}")
+if(WIN32)
+set(CPACK_ALL_INSTALL_TYPES Full Developer)
+set(CPACK_COMPONENT_LIBRARIES_INSTALL_TYPES Developer Full)
+set(CPACK_COMPONENT_HEADERS_INSTALL_TYPES Developer Full)
+set(CPACK_COMPONENT_APPLICATIONS_INSTALL_TYPES Full)
+
+else()
 set(CPACK_SOURCE_GENERATOR "TGZ")
+endif(WIN32)
 set(CPACK_SOURCE_PACKAGE_FILE_NAME "${ASSEMBLY_BASE_NAME}-source")
 set(CPACK_SOURCE_IGNORE_FILES "/docs/generated/;${CMAKE_SOURCE_DIR}/build/;~$;${CPACK_SOURCE_IGNORE_FILES};${CMAKE_SOURCE_DIR}/.git/;${CMAKE_SOURCE_DIR}/.idea/;${CMAKE_SOURCE_DIR}/cmake-build-debug/;${CMAKE_SOURCE_DIR}/extensions/expression-language/Scanner.h;${CMAKE_SOURCE_DIR}/extensions/expression-language/Scanner.cpp;${CMAKE_SOURCE_DIR}/extensions/expression-language/Parser.cpp;${CMAKE_SOURCE_DIR}/extensions/expression-language/Parser.hpp;${CMAKE_SOURCE_DIR}/extensions/expression-language/scanner.hh;${CMAKE_SOURCE_DIR}/extensions/expression-language/position.hh;${CMAKE_SOURCE_DIR}/extensions/expression-language/location.hh;${CMAKE_SOURCE_DIR}/extensions/expression-language/stack.hh")
 
@@ -549,14 +565,17 @@ install(FILES LICENSE README.md NOTICE
         DESTINATION .
         COMPONENT bin)
 
-set(CPACK_GENERATOR "TGZ")
+
 set(CPACK_COMPONENT_INCLUDE_TOPLEVEL_DIRECTORY 1)
 set(CPACK_PACKAGE_DESCRIPTION_SUMMARY "Apache NiFi MiNiFi C++ version ${VERSION}")
 set(CPACK_PACKAGE_VENDOR "Apache NiFi")
 set(CPACK_PACKAGE_DESCRIPTION_FILE "${CMAKE_CURRENT_SOURCE_DIR}/README.md")
 set(CPACK_RESOURCE_FILE_LICENSE "${CMAKE_CURRENT_SOURCE_DIR}/LICENSE")
 set(CPACK_PACKAGE_FILE_NAME "${ASSEMBLY_BASE_NAME}")
+if(NOT WIN32)
+set(CPACK_GENERATOR "TGZ")
 set(CPACK_BINARY_TGZ, "ON")
+endif()
 set(CPACK_ARCHIVE_COMPONENT_INSTALL ON)
 set(CPACK_RDKAFKA_COMPONENT_INSTALL ON)
 set(CPACK_MQTT_COMPONENT_INSTALL ON)
diff --git a/CMakeSettings.json b/CMakeSettings.json
index a995e250..e58a3629 100644
--- a/CMakeSettings.json
+++ b/CMakeSettings.json
@@ -1,7 +1,7 @@
 {
   // See https://go.microsoft.com//fwlink//?linkid=834763 for more information about this file.
   "configurations": [
-       {
+    {
       "name": "x64-Debug",
       "generator": "Visual Studio 14 2015 Win64",
       "configurationType": "Debug",
@@ -10,73 +10,73 @@
       "installRoot": "${env.USERPROFILE}\\CMakeBuilds\\${workspaceHash}\\install\\${name}",
       "cmakeCommandArgs": "",
       "buildCommandArgs": "-v:minimal -m:8",
-        "variables": [
-          {
-            "name": "OPENSSL_OFF",
-            "value": "OFF"
-          },
-          {
-            "name": "USE_SYSTEM_OPENSSL",
-            "value": "ON"
-          },
-          {
-            "name": "OPENSSL_ROOT_DIR",
-            "value": "C:\\OpenSSL-Win64"
-          },
-          {
-            "name": "USE_SYSTEM_ZLIB",
-            "value": "OFF"
-          },
-          {
-            "name": "USE_SYSTEM_CURL",
-            "value": "OFF"
-          },
-          {
-            "name": "USE_SYSTEM_UUID",
-            "value": "OFF"
-          },
-          {
-            "name": "FORCE_WINDOWS",
-            "value": "ON"
-          },
-          {
-            "name": "DISABLE_ROCKSDB",
-            "value": "FALSE"
-          },
-          {
-            "name": "ENABLE_WEL",
-            "value": "TRUE"
-          },
-          {
-            "name": "ENABLE_LIBRDKAFKA",
-            "value": "OFF"
-          },
-          {
-            "name": "DISABLE_CURL",
-            "value": "OFF"
-          },
-          {
-            "name": "DISABLE_LIBARCHIVE",
-            "value": "TRUE"
-          },
-          {
-            "name": "DISABLE_SCRIPTING",
-            "value": "TRUE"
-          },
-          {
-            "name": "EXCLUDE_BOOST",
-            "value": "TRUE"
-          },
-          {
-            "name": "DISABLE_EXPRESSION_LANGUAGE",
-            "value": "TRUE"
-          },
-          {
-            "name": "SKIP_TESTS",
-            "value": "TRUE"
-          }
+      "variables": [
+        {
+          "name": "OPENSSL_OFF",
+          "value": "OFF"
+        },
+        {
+          "name": "USE_SYSTEM_OPENSSL",
+          "value": "ON"
+        },
+        {
+          "name": "OPENSSL_ROOT_DIR",
+          "value": "C:\\OpenSSL-Win64"
+        },
+        {
+          "name": "USE_SYSTEM_ZLIB",
+          "value": "OFF"
+        },
+        {
+          "name": "USE_SYSTEM_CURL",
+          "value": "OFF"
+        },
+        {
+          "name": "USE_SYSTEM_UUID",
+          "value": "OFF"
+        },
+        {
+          "name": "FORCE_WINDOWS",
+          "value": "ON"
+        },
+        {
+          "name": "DISABLE_ROCKSDB",
+          "value": "FALSE"
+        },
+        {
+          "name": "ENABLE_WEL",
+          "value": "TRUE"
+        },
+        {
+          "name": "ENABLE_LIBRDKAFKA",
+          "value": "OFF"
+        },
+        {
+          "name": "DISABLE_CURL",
+          "value": "OFF"
+        },
+        {
+          "name": "DISABLE_LIBARCHIVE",
+          "value": "TRUE"
+        },
+        {
+          "name": "DISABLE_SCRIPTING",
+          "value": "TRUE"
+        },
+        {
+          "name": "EXCLUDE_BOOST",
+          "value": "TRUE"
+        },
+        {
+          "name": "DISABLE_EXPRESSION_LANGUAGE",
+          "value": "TRUE"
+        },
+        {
+          "name": "SKIP_TESTS",
+          "value": "TRUE"
+        }
 
-        ],
+      ],
       "ctestCommandArgs": ""
     },
     {
@@ -87,16 +87,29 @@
       "buildRoot": "${env.USERPROFILE}\\CMakeBuilds\\${workspaceHash}\\build\\${name}",
       "installRoot": "${env.USERPROFILE}\\CMakeBuilds\\${workspaceHash}\\install\\${name}",
       "cmakeCommandArgs": "",
-      "buildCommandArgs": "-v",
+      "buildCommandArgs": "-v -j8",
+      "$" { "env.USERPROFILE" },
       "variables": [
         {
           "name": "OPENSSL_OFF",
+          "value": "OFF"
+        },
+        {
+          "name": "USE_SYSTEM_OPENSSL",
           "value": "ON"
         },
+        {
+          "name": "OPENSSL_ROOT_DIR",
+          "value": "C:\\OpenSSL-Win64"
+        },
         {
           "name": "USE_SYSTEM_ZLIB",
           "value": "OFF"
         },
+        {
+          "name": "DISABLE_CONTROLLER",
+          "value": "TRUE"
+        },
         {
           "name": "USE_SYSTEM_CURL",
           "value": "OFF"
@@ -145,7 +158,6 @@
           "name": "SKIP_TESTS",
           "value": "TRUE"
         }
-
       ],
       "ctestCommandArgs": ""
     }
diff --git a/extensions/civetweb/processors/ListenHTTP.cpp b/extensions/civetweb/processors/ListenHTTP.cpp
index 210ae1c3..6f00776a 100644
--- a/extensions/civetweb/processors/ListenHTTP.cpp
+++ b/extensions/civetweb/processors/ListenHTTP.cpp
@@ -26,23 +26,29 @@ namespace nifi {
 namespace minifi {
 namespace processors {
 
-core::Property ListenHTTP::BasePath("Base Path", "Base path for incoming connections", "contentListener");
-core::Property ListenHTTP::Port("Listening Port", "The Port to listen on for incoming connections", "");
-core::Property ListenHTTP::AuthorizedDNPattern
-    ("Authorized DN Pattern", "A Regular Expression to apply against the Distinguished Name of incoming"
-         " connections. If the Pattern does not match the DN, the connection will be refused.",
-     ".*");
-core::Property ListenHTTP::SSLCertificate
-    ("SSL Certificate", "File containing PEM-formatted file including TLS/SSL certificate and key", "");
-core::Property ListenHTTP::SSLCertificateAuthority
-    ("SSL Certificate Authority", "File containing trusted PEM-formatted certificates", "");
-core::Property
-    ListenHTTP::SSLVerifyPeer("SSL Verify Peer", "Whether or not to verify the client's certificate (yes/no)", "no");
-core::Property ListenHTTP::SSLMinimumVersion
-    ("SSL Minimum Version", "Minimum TLS/SSL version allowed (SSL2, SSL3, TLS1.0, TLS1.1, TLS1.2)", "SSL2");
-core::Property ListenHTTP::HeadersAsAttributesRegex("HTTP Headers to receive as Attributes (Regex)",
-                                                    "Specifies the Regular Expression that determines the names of HTTP Headers that"
-                                                        " should be passed along as FlowFile attributes",
+core::Property ListenHTTP::BasePath(
+    core::PropertyBuilder::createProperty("Base Path")->withDescription("Base path for incoming connections")->isRequired(false)->withDefaultValue<std::string>("contentListener")->build());
+
+core::Property ListenHTTP::Port(
+    core::PropertyBuilder::createProperty("Listening Port")->withDescription("The Port to listen on for incoming connections")->isRequired(true)->withDefaultValue<int>(
+        80, core::StandardValidators::PORT_VALIDATOR())->build());
+
+core::Property ListenHTTP::AuthorizedDNPattern("Authorized DN Pattern", "A Regular Expression to apply against the Distinguished Name of incoming"
+                                               " connections. If the Pattern does not match the DN, the connection will be refused.",
+                                               ".*");
+core::Property ListenHTTP::SSLCertificate("SSL Certificate", "File containing PEM-formatted file including TLS/SSL certificate and key", "");
+core::Property ListenHTTP::SSLCertificateAuthority("SSL Certificate Authority", "File containing trusted PEM-formatted certificates", "");
+
+core::Property ListenHTTP::SSLVerifyPeer(
+    core::PropertyBuilder::createProperty("SSL Verify Peer")->withDescription("Whether or not to verify the client's certificate (yes/no)")->isRequired(false)->withAllowableValue<std::string>("yes")
+        ->withAllowableValue("no")->withDefaultValue("no")->build());
+
+core::Property ListenHTTP::SSLMinimumVersion(
+    core::PropertyBuilder::createProperty("SSL Minimum Version")->withDescription("Minimum TLS/SSL version allowed (SSL2, SSL3, TLS1.0, TLS1.1, TLS1.2)")->isRequired(false)
+        ->withAllowableValue<std::string>("SSL2")->withAllowableValue("SSL3")->withAllowableValue("TLS1.0")->withAllowableValue("TLS1.1")->withAllowableValue("TLS1.2")->withDefaultValue("SSL2")->build());
+
+core::Property ListenHTTP::HeadersAsAttributesRegex("HTTP Headers to receive as Attributes (Regex)", "Specifies the Regular Expression that determines the names of HTTP Headers that"
+                                                    " should be passed along as FlowFile attributes",
                                                     "");
 
 core::Relationship ListenHTTP::Success("success", "All files are routed to success");
@@ -71,10 +77,8 @@ void ListenHTTP::onSchedule(core::ProcessContext *context, core::ProcessSessionF
   std::string basePath;
 
   if (!context->getProperty(BasePath.getName(), basePath)) {
-    logger_->log_info("%s attribute is missing, so default value of %s will be used",
-                      BasePath.getName(),
-                      BasePath.getValue());
-    basePath = BasePath.getValue();
+    logger_->log_info("%s attribute is missing, so default value of %s will be used", BasePath.getName(), BasePath.getValue().to_string());
+    basePath = BasePath.getValue().to_string();
   }
 
   basePath.insert(0, "/");
@@ -104,8 +108,7 @@ void ListenHTTP::onSchedule(core::ProcessContext *context, core::ProcessSessionF
   std::string sslMinVer;
 
   if (!sslCertFile.empty()) {
-    if (context->getProperty(SSLCertificateAuthority.getName(), sslCertAuthorityFile)
-        && !sslCertAuthorityFile.empty()) {
+    if (context->getProperty(SSLCertificateAuthority.getName(), sslCertAuthorityFile) && !sslCertAuthorityFile.empty()) {
       logger_->log_debug("ListenHTTP using %s: %s", SSLCertificateAuthority.getName(), sslCertAuthorityFile);
     }
 
@@ -126,17 +129,13 @@ void ListenHTTP::onSchedule(core::ProcessContext *context, core::ProcessSessionF
 
   std::string headersAsAttributesPattern;
 
-  if (context->getProperty(HeadersAsAttributesRegex.getName(), headersAsAttributesPattern)
-      && !headersAsAttributesPattern.empty()) {
+  if (context->getProperty(HeadersAsAttributesRegex.getName(), headersAsAttributesPattern) && !headersAsAttributesPattern.empty()) {
     logger_->log_debug("ListenHTTP using %s: %s", HeadersAsAttributesRegex.getName(), headersAsAttributesPattern);
   }
 
   auto numThreads = getMaxConcurrentTasks();
 
-  logger_->log_info("ListenHTTP starting HTTP server on port %s and path %s with %d threads",
-                    listeningPort,
-                    basePath,
-                    numThreads);
+  logger_->log_info("ListenHTTP starting HTTP server on port %s and path %s with %d threads", listeningPort, basePath, numThreads);
 
   // Initialize web server
   std::vector<std::string> options;
@@ -190,11 +189,7 @@ void ListenHTTP::onSchedule(core::ProcessContext *context, core::ProcessSessionF
   }
 
   server_.reset(new CivetServer(options));
-  handler_.reset(new Handler(basePath,
-                             context,
-                             sessionFactory,
-                             std::move(authDNPattern),
-                             std::move(headersAsAttributesPattern)));
+  handler_.reset(new Handler(basePath, context, sessionFactory, std::move(authDNPattern), std::move(headersAsAttributesPattern)));
   server_->addHandler(basePath, handler_.get());
 }
 
@@ -215,13 +210,12 @@ void ListenHTTP::onTrigger(core::ProcessContext *context, core::ProcessSession *
   if (type == "response_body") {
 
     if (handler_) {
-      struct response_body response{"", "", ""};
+      struct response_body response { "", "", "" };
       ResponseBodyReadCallback cb(&response.body);
       flow_file->getAttribute("filename", response.uri);
       flow_file->getAttribute("mime.type", response.mime_type);
       if (response.mime_type.empty()) {
-        logger_->log_warn("Using default mime type of application/octet-stream for response body file: %s",
-                          response.uri);
+        logger_->log_warn("Using default mime type of application/octet-stream for response body file: %s", response.uri);
         response.mime_type = "application/octet-stream";
       }
       session->read(flow_file, &cb);
@@ -232,11 +226,7 @@ void ListenHTTP::onTrigger(core::ProcessContext *context, core::ProcessSession *
   session->remove(flow_file);
 }
 
-ListenHTTP::Handler::Handler(std::string base_uri,
-                             core::ProcessContext *context,
-                             core::ProcessSessionFactory *session_factory,
-                             std::string &&auth_dn_regex,
-                             std::string &&header_as_attrs_regex)
+ListenHTTP::Handler::Handler(std::string base_uri, core::ProcessContext *context, core::ProcessSessionFactory *session_factory, std::string &&auth_dn_regex, std::string &&header_as_attrs_regex)
     : base_uri_(std::move(base_uri)),
       auth_dn_regex_(std::move(auth_dn_regex)),
       headers_as_attrs_regex_(std::move(header_as_attrs_regex)),
@@ -246,14 +236,12 @@ ListenHTTP::Handler::Handler(std::string base_uri,
 }
 
 void ListenHTTP::Handler::send_error_response(struct mg_connection *conn) {
-  mg_printf(conn,
-            "HTTP/1.1 500 Internal Server Error\r\n"
-                "Content-Type: text/html\r\n"
-                "Content-Length: 0\r\n\r\n");
+  mg_printf(conn, "HTTP/1.1 500 Internal Server Error\r\n"
+            "Content-Type: text/html\r\n"
+            "Content-Length: 0\r\n\r\n");
 }
 
-void ListenHTTP::Handler::set_header_attributes(const mg_request_info *req_info,
-                                                const std::shared_ptr<FlowFileRecord> &flow_file) const {
+void ListenHTTP::Handler::set_header_attributes(const mg_request_info *req_info, const std::shared_ptr<FlowFileRecord> &flow_file) const {
   // Add filename from "filename" header value (and pattern headers)
   for (int i = 0; i < req_info->num_headers; i++) {
     auto header = &req_info->http_headers[i];
@@ -324,8 +312,8 @@ bool ListenHTTP::Handler::auth_request(mg_connection *conn, const mg_request_inf
   if (req_info->is_ssl && req_info->client_cert != nullptr) {
     if (!std::regex_match(req_info->client_cert->subject, auth_dn_regex_)) {
       mg_printf(conn, "HTTP/1.1 403 Forbidden\r\n"
-          "Content-Type: text/html\r\n"
-          "Content-Length: 0\r\n\r\n");
+                "Content-Type: text/html\r\n"
+                "Content-Length: 0\r\n\r\n");
       logger_->log_warn("ListenHTTP client DN not authorized: %s", req_info->client_cert->subject);
       authorized = false;
     }
@@ -376,7 +364,7 @@ void ListenHTTP::Handler::write_body(mg_connection *conn, const mg_request_info
   const auto &request_uri_str = std::string(req_info->request_uri);
 
   if (request_uri_str.size() > base_uri_.size() + 1) {
-    struct response_body response{};
+    struct response_body response { };
 
     {
       // Attempt to minimize time holding mutex (it would be nice to have a lock-free concurrent map here)
@@ -389,9 +377,7 @@ void ListenHTTP::Handler::write_body(mg_connection *conn, const mg_request_info
     }
 
     if (!response.body.empty()) {
-      logger_->log_debug("Writing response body of %lu bytes for URI: %s",
-                         response.body.size(),
-                         req_info->request_uri);
+      logger_->log_debug("Writing response body of %lu bytes for URI: %s", response.body.size(), req_info->request_uri);
       mg_printf(conn, "Content-type: ");
       mg_printf(conn, "%s", response.mime_type.c_str());
       mg_printf(conn, "\r\n");
diff --git a/extensions/gps/GetGPS.cpp b/extensions/gps/GetGPS.cpp
index 08a5d160..484a4123 100644
--- a/extensions/gps/GetGPS.cpp
+++ b/extensions/gps/GetGPS.cpp
@@ -38,6 +38,7 @@
 #include "GetGPS.h"
 #include "core/ProcessContext.h"
 #include "core/ProcessSession.h"
+#include "core/PropertyValidation.h"
 
 #define policy_t gps_policy_t
 #include <libgpsmm.h>
@@ -52,9 +53,11 @@ namespace processors {
 
 const std::string GetGPS::ProcessorName("GetGPS");
 core::Relationship GetGPS::Success("success", "All files are routed to success");
-core::Property GetGPS::GPSDHost("GPSD Host", "The host running the GPSD daemon", "localhost");
-core::Property GetGPS::GPSDPort("GPSD Port", "The GPSD daemon port", "2947");
-core::Property GetGPS::GPSDWaitTime("GPSD Wait Time", "Timeout value for waiting for data from the GPSD instance", "50000000");
+core::Property GetGPS::GPSDHost(core::PropertyBuilder::createProperty("GPSD Host")->withDescription("The host running the GPSD daemon")->withDefaultValue<std::string>("localhost")->build());
+core::Property GetGPS::GPSDPort(
+    core::PropertyBuilder::createProperty("GPSD Port")->withDescription("The GPSD daemon port")->withDefaultValue<int64_t>(2947, core::StandardValidators::PORT_VALIDATOR())->build());
+core::Property GetGPS::GPSDWaitTime(
+    core::PropertyBuilder::createProperty("GPSD Wait Time")->withDescription("Timeout value for waiting for data from the GPSD instance")->withDefaultValue<uint64_t>(50000000)->build());
 void GetGPS::initialize() {
   //! Set the supported properties
   std::set<core::Property> properties;
diff --git a/extensions/http-curl/processors/InvokeHTTP.cpp b/extensions/http-curl/processors/InvokeHTTP.cpp
index 188b40fa..a2cd3328 100644
--- a/extensions/http-curl/processors/InvokeHTTP.cpp
+++ b/extensions/http-curl/processors/InvokeHTTP.cpp
@@ -66,7 +66,10 @@ core::Property InvokeHTTP::URL(
         true)->build());
 core::Property InvokeHTTP::ConnectTimeout("Connection Timeout", "Max wait time for connection to remote service.", "5 secs");
 core::Property InvokeHTTP::ReadTimeout("Read Timeout", "Max wait time for response from remote service.", "15 secs");
-core::Property InvokeHTTP::DateHeader("Include Date Header", "Include an RFC-2616 Date header in the request.", "True");
+
+core::Property InvokeHTTP::DateHeader(
+    core::PropertyBuilder::createProperty("Include Date Header")->withDescription("Include an RFC-2616 Date header in the request.")->isRequired(false)->withDefaultValue<bool>(true)->build());
+
 core::Property InvokeHTTP::FollowRedirects("Follow Redirects", "Follow HTTP redirects issued by remote server.", "True");
 core::Property InvokeHTTP::AttributesToSend("Attributes to Send", "Regular expression that defines which attributes to send as HTTP"
                                             " headers in the request. If not defined, no attributes are sent as headers.",
diff --git a/extensions/mqtt/protocol/PayloadSerializer.h b/extensions/mqtt/protocol/PayloadSerializer.h
index 27b33622..fbc5d382 100644
--- a/extensions/mqtt/protocol/PayloadSerializer.h
+++ b/extensions/mqtt/protocol/PayloadSerializer.h
@@ -19,9 +19,9 @@
 #ifndef EXTENSIONS_MQTT_PROTOCOL_PAYLOADSERIALIZER_H_
 #define EXTENSIONS_MQTT_PROTOCOL_PAYLOADSERIALIZER_H_
 
+#include "core/state/Value.h"
 #include "c2/C2Protocol.h"
 #include "io/BaseStream.h"
-#include "core/state/Value.h"
 namespace org {
 namespace apache {
 namespace nifi {
diff --git a/extensions/pcap/CapturePacket.cpp b/extensions/pcap/CapturePacket.cpp
index 3869452e..6a07e2e7 100644
--- a/extensions/pcap/CapturePacket.cpp
+++ b/extensions/pcap/CapturePacket.cpp
@@ -52,10 +52,11 @@ namespace minifi {
 namespace processors {
 
 std::shared_ptr<utils::IdGenerator> CapturePacket::id_generator_ = utils::IdGenerator::getIdGenerator();
-core::Property CapturePacket::BaseDir("Base Directory", "Scratch directory for PCAP files", "/tmp/");
-core::Property CapturePacket::BatchSize("Batch Size", "The number of packets to combine within a given PCAP", "50");
+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::CaptureBluetooth("Capture Bluetooth", "True indicates that we support bluetooth interfaces", "false");
+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";
 
diff --git a/libminifi/include/FlowController.h b/libminifi/include/FlowController.h
index 9309b4f2..a1e9fd16 100644
--- a/libminifi/include/FlowController.h
+++ b/libminifi/include/FlowController.h
@@ -113,7 +113,7 @@ class FlowController : public core::controller::ControllerServiceProvider, publi
   }
 
   // Load flow xml from disk, after that, create the root process group and its children, initialize the flows
-  virtual void load();
+  virtual void load(const std::shared_ptr<core::ProcessGroup> &root = nullptr, bool reload = false);
 
   // Whether the Flow Controller is start running
   virtual bool isRunning() {
@@ -175,21 +175,27 @@ class FlowController : public core::controller::ControllerServiceProvider, publi
   bool applyConfiguration(const std::string &source, const std::string &configurePayload);
 
   // get name
-  std::string getName() const{
+  std::string getName() const {
     if (root_ != nullptr)
       return root_->getName();
     else
       return "";
   }
 
-  virtual std::string getComponentName() {
+  virtual std::string getComponentName() const {
     return "FlowController";
   }
 
+  virtual std::string getComponentUUID() const {
+    utils::Identifier ident;
+    root_->getUUID(ident);
+    return ident.to_string();
+  }
+
   // get version
   virtual std::string getVersion() {
     if (root_ != nullptr)
-      return std::to_string( root_->getVersion() );
+      return std::to_string(root_->getVersion());
     else
       return "0";
   }
@@ -200,8 +206,7 @@ class FlowController : public core::controller::ControllerServiceProvider, publi
    * @param id service identifier
    * @param firstTimeAdded first time this CS was added
    */
-  virtual std::shared_ptr<core::controller::ControllerServiceNode> createControllerService(const std::string &type, const std::string &id,
-  bool firstTimeAdded);
+  virtual std::shared_ptr<core::controller::ControllerServiceNode> createControllerService(const std::string &type, const std::string &id, bool firstTimeAdded);
 
   /**
    * controller service provider
@@ -361,6 +366,7 @@ class FlowController : public core::controller::ControllerServiceProvider, publi
   std::string configuration_filename_;
 
   std::atomic<bool> c2_initialized_;
+  std::atomic<bool> flow_update_;
   std::atomic<bool> c2_enabled_;
   // Whether it has already been initialized (load the flow XML already)
   std::atomic<bool> initialized_;
@@ -398,18 +404,18 @@ class FlowController : public core::controller::ControllerServiceProvider, publi
 
   std::mutex metrics_mutex_;
   // root_nodes cache
-    std::map<std::string, std::shared_ptr<state::response::ResponseNode>> root_response_nodes_;
+  std::map<std::string, std::shared_ptr<state::response::ResponseNode>> root_response_nodes_;
   // metrics cache
   std::map<std::string, std::shared_ptr<state::response::ResponseNode>> device_information_;
 
   // metrics cache
   std::map<std::string, std::shared_ptr<state::response::ResponseNode>> component_metrics_;
 
-  std::map<uint8_t, std::vector<std::shared_ptr<state::response::ResponseNode>>>component_metrics_by_id_;
+  std::map<uint8_t, std::vector<std::shared_ptr<state::response::ResponseNode>>> component_metrics_by_id_;
   // metrics last run
   std::chrono::steady_clock::time_point last_metrics_capture_;
 
-private:
+ private:
   std::shared_ptr<logging::Logger> logger_;
   std::string serial_number_;
   static std::shared_ptr<utils::IdGenerator> id_generator_;
diff --git a/libminifi/include/SchedulingAgent.h b/libminifi/include/SchedulingAgent.h
index 925efdb1..3b94d0ee 100644
--- a/libminifi/include/SchedulingAgent.h
+++ b/libminifi/include/SchedulingAgent.h
@@ -128,7 +128,8 @@ class SchedulingAgent {
      * To facilitate traces we cannot use daemon threads -- this could potentially cause blocking on I/O; however, it's a better path
      * to be able to debug why an agent doesn't work and still allow a restart via updates in these cases.
      */
-    auto pool = utils::ThreadPool<uint64_t>(configure_->getInt(Configure::nifi_flow_engine_threads, 2), false, controller_service_provider, "SchedulingAgent");
+    auto csThreads = configure_->getInt(Configure::nifi_flow_engine_threads, 2);
+    auto pool = utils::ThreadPool<uint64_t>(csThreads, false, controller_service_provider, "SchedulingAgent");
     thread_pool_ = std::move(pool);
     thread_pool_.start();
   }
diff --git a/libminifi/include/agent/build_description.h b/libminifi/include/agent/build_description.h
index 4d6a1e6a..8e00f601 100644
--- a/libminifi/include/agent/build_description.h
+++ b/libminifi/include/agent/build_description.h
@@ -85,7 +85,6 @@ class BuildDescription {
         utils::StringUtils::replaceAll(classDescriptionName,"::",".");
         ClassDescription description(classDescriptionName);
         if (nullptr != component) {
-
           auto processor = std::dynamic_pointer_cast<core::Processor>(obj) ;
           bool is_processor = processor != nullptr;
           bool is_controller_service = LIKELY(is_processor == true) ? false : std::dynamic_pointer_cast<core::controller::ControllerService>(obj) != nullptr;
diff --git a/libminifi/include/c2/C2Agent.h b/libminifi/include/c2/C2Agent.h
index cc575a21..c3945fbf 100644
--- a/libminifi/include/c2/C2Agent.h
+++ b/libminifi/include/c2/C2Agent.h
@@ -26,9 +26,9 @@
 #include <thread>
 
 #include "../core/state/nodes/MetricsBase.h"
+#include "../core/state/Value.h"
 #include "core/state/UpdateController.h"
 #include "controllers/UpdatePolicyControllerService.h"
-#include "core/state/Value.h"
 #include "C2Payload.h"
 #include "C2Trigger.h"
 #include "C2Protocol.h"
@@ -118,7 +118,7 @@ class C2Agent : public state::UpdateController, public state::response::Response
    * @param name name of this metric
    * @param metrics metrics to include.
    */
-  void serializeMetrics(C2Payload &parent_payload, const std::string &name, const std::vector<state::response::SerializedResponseNode> &metrics, bool is_container = false);
+  void serializeMetrics(C2Payload &parent_payload, const std::string &name, const std::vector<state::response::SerializedResponseNode> &metrics, bool is_container = false, bool is_collapsible = true);
 
   /**
    * Extract the payload
@@ -165,6 +165,11 @@ class C2Agent : public state::UpdateController, public state::response::Response
    */
   void handle_describe(const C2ContentResponse &resp);
 
+  /**
+   * Updates a property
+   */
+  bool update_property(const std::string &property_name, const std::string &property_value,  bool persist = false);
+
   std::timed_mutex metrics_mutex_;
   std::map<std::string, std::shared_ptr<state::response::ResponseNode>> metrics_map_;
 
diff --git a/libminifi/include/c2/C2Payload.h b/libminifi/include/c2/C2Payload.h
index cbe2cb1f..f804c2b4 100644
--- a/libminifi/include/c2/C2Payload.h
+++ b/libminifi/include/c2/C2Payload.h
@@ -21,8 +21,9 @@
 #include <memory>
 #include <string>
 #include <map>
+
+#include "../core/state/Value.h"
 #include "core/state/UpdateController.h"
-#include "core/state/Value.h"
 
 namespace org {
 namespace apache {
@@ -113,9 +114,9 @@ class C2Payload : public state::Update {
 
   C2Payload(Operation op, state::UpdateState state, bool resp = false, bool isRaw = false);
 
-  C2Payload(const C2Payload &other);
+  C2Payload(const C2Payload &other) = default;
 
-  C2Payload(const C2Payload &&other);
+  C2Payload(C2Payload &&other) = default;
 
   void setIdentifier(const std::string &ident);
 
@@ -147,7 +148,7 @@ class C2Payload : public state::Update {
   /**
    * Add a content response to this payload.
    */
-  void addContent(const C2ContentResponse &&content);
+  void addContent(const C2ContentResponse &&content, bool collapsible = true);
 
   /**
    * Determines if this object contains raw data.
@@ -180,6 +181,14 @@ class C2Payload : public state::Update {
    */
   void addPayload(const C2Payload &&payload);
 
+  bool isCollapsible() const {
+    return is_collapsible_;
+  }
+
+  void setCollapsible(bool is_collapsible) {
+    is_collapsible_ = is_collapsible;
+  }
+
   bool isContainer() const {
     return is_container_;
   }
@@ -192,8 +201,8 @@ class C2Payload : public state::Update {
    */
   const std::vector<C2Payload> &getNestedPayloads() const;
 
-  C2Payload &operator=(const C2Payload &&other);
-  C2Payload &operator=(const C2Payload &other);
+  C2Payload &operator=(C2Payload &&other) = default;
+  C2Payload &operator=(const C2Payload &other) = default;
 
   inline bool operator==(const C2Payload &rhs) const {
     if (op_ != rhs.op_) {
@@ -245,6 +254,8 @@ class C2Payload : public state::Update {
 
   bool is_container_;
 
+  bool is_collapsible_;
+
 };
 
 } /* namesapce c2 */
diff --git a/libminifi/include/c2/protocols/RESTProtocol.h b/libminifi/include/c2/protocols/RESTProtocol.h
index 53e3f4f3..251622c5 100644
--- a/libminifi/include/c2/protocols/RESTProtocol.h
+++ b/libminifi/include/c2/protocols/RESTProtocol.h
@@ -47,6 +47,12 @@ namespace c2 {
  * will encompass other protocols the context of its meaning here simply translates into POST and GET respectively.
  *
  */
+
+struct ValueObject{
+  std::string name;
+  std::vector<rapidjson::Value*> values;
+};
+
 class RESTProtocol {
  public:
   RESTProtocol()
diff --git a/libminifi/include/controllers/LinuxPowerManagementService.h b/libminifi/include/controllers/LinuxPowerManagementService.h
index 9e729b11..6b348e9b 100644
--- a/libminifi/include/controllers/LinuxPowerManagementService.h
+++ b/libminifi/include/controllers/LinuxPowerManagementService.h
@@ -136,7 +136,7 @@ class LinuxPowerManagerService : public ThreadManagementService {
   std::shared_ptr<logging::Logger> logger_;
 };
 
-REGISTER_RESOURCE(LinuxPowerManagerService, "Linux power management service that enables control of power usage in the agent through Linux power management information");
+REGISTER_RESOURCE(LinuxPowerManagerService, "Linux power management service that enables control of power usage in the agent through Linux power management information. Use name \"ThreadPoolManager\" to throttle battery consumption");
 
 } /* namespace controllers */
 } /* namespace minifi */
diff --git a/libminifi/include/core/ConfigurableComponent.h b/libminifi/include/core/ConfigurableComponent.h
index d7929a92..d72fc83c 100644
--- a/libminifi/include/core/ConfigurableComponent.h
+++ b/libminifi/include/core/ConfigurableComponent.h
@@ -54,7 +54,8 @@ class ConfigurableComponent {
    * @param value value passed in by reference
    * @return result of getting property.
    */
-  bool getProperty(const std::string name, std::string &value) const;
+  template<typename T>
+  bool getProperty(const std::string name, T &value) const;
 
   /**
    * Provides a reference for the property.
@@ -81,6 +82,14 @@ class ConfigurableComponent {
    */
   bool setProperty(Property &prop, std::string value);
 
+  /**
+     * Sets the property using the provided name
+     * @param property name
+     * @param value property value.
+     * @return whether property was set or not
+     */
+  bool setProperty(Property &prop, PropertyValue &value);
+
   /**
    * Sets supported properties for the ConfigurableComponent
    * @param supported properties
@@ -191,6 +200,27 @@ class ConfigurableComponent {
 
 };
 
+template<typename T>
+bool ConfigurableComponent::getProperty(const std::string name, T &value) const{
+  std::lock_guard<std::mutex> lock(configuration_mutex_);
+
+   auto &&it = properties_.find(name);
+   if (it != properties_.end()) {
+     Property item = it->second;
+     value = static_cast<T>(item.getValue());
+     if (item.getValue().getValue() != nullptr){
+       logger_->log_debug("Component %s property name %s value %s", name, item.getName(), item.getValue().to_string());
+       return true;
+     }
+     else{
+       logger_->log_debug("Component %s property name %s, empty value", name, item.getName());
+       return false;
+     }
+   } else {
+     return false;
+   }
+}
+
 } /* namespace core */
 } /* namespace minifi */
 } /* namespace nifi */
diff --git a/libminifi/include/core/FlowConfiguration.h b/libminifi/include/core/FlowConfiguration.h
index 9d2562d4..b816c1af 100644
--- a/libminifi/include/core/FlowConfiguration.h
+++ b/libminifi/include/core/FlowConfiguration.h
@@ -70,7 +70,11 @@ class FlowConfiguration : public CoreComponent {
         logger_(logging::LoggerFactory<FlowConfiguration>::getLogger()) {
     controller_services_ = std::make_shared<core::controller::ControllerServiceMap>();
     service_provider_ = std::make_shared<core::controller::StandardControllerServiceProvider>(controller_services_, nullptr, configuration);
-    flow_version_ = std::make_shared<state::response::FlowVersion>("", "default", "");
+    std::string flowUrl = "", bucket_id = "default", flowId = "";
+    configuration->get(Configure::nifi_c2_flow_id, flowId);
+    configuration->get(Configure::nifi_c2_flow_url, flowUrl);
+    flow_version_ = std::make_shared<state::response::FlowVersion>(flowUrl, bucket_id, flowId);
+
     // it is okay if this has already been called
     initialize_static_functions();
   }
diff --git a/libminifi/include/core/ProcessContext.h b/libminifi/include/core/ProcessContext.h
index 352ef382..39d1d523 100644
--- a/libminifi/include/core/ProcessContext.h
+++ b/libminifi/include/core/ProcessContext.h
@@ -66,9 +66,12 @@ class ProcessContext : public controller::ControllerServiceLookup {
   std::shared_ptr<ProcessorNode> getProcessorNode() const {
     return processor_node_;
   }
-  bool getProperty(const std::string &name, std::string &value) const {
-    return processor_node_->getProperty(name, value);
+
+  template<typename T>
+  bool getProperty(const std::string &name, T &value) const {
+    return getPropertyImp<typename std::common_type<T>::type>(name, value);
   }
+
   bool getProperty(const Property &property, std::string &value, const std::shared_ptr<FlowFile> &flow_file);
   bool getDynamicProperty(const std::string &name, std::string &value) const {
     return processor_node_->getDynamicProperty(name, value);
@@ -173,6 +176,11 @@ class ProcessContext : public controller::ControllerServiceLookup {
 
  private:
 
+  template<typename T>
+  bool getPropertyImp(const std::string &name, T &value) const {
+    return processor_node_->getProperty<typename std::common_type<T>::type>(name, value);
+  }
+
   // controller service provider.
   std::shared_ptr<controller::ControllerServiceProvider> controller_service_provider_;
   // repository shared pointer.
diff --git a/libminifi/include/core/ProcessorNode.h b/libminifi/include/core/ProcessorNode.h
index 9c574b95..540f25a9 100644
--- a/libminifi/include/core/ProcessorNode.h
+++ b/libminifi/include/core/ProcessorNode.h
@@ -59,12 +59,13 @@ class ProcessorNode : public ConfigurableComponent, public Connectable {
    * @param value value passed in by reference
    * @return result of getting property.
    */
-  bool getProperty(const std::string &name, std::string &value) const {
+  template<typename T>
+  bool getProperty(const std::string &name, T &value) {
     const std::shared_ptr<ConfigurableComponent> processor_cast = std::dynamic_pointer_cast<ConfigurableComponent>(processor_);
     if (nullptr != processor_cast)
-      return processor_cast->getProperty(name, value);
+      return processor_cast->getProperty<T>(name, value);
     else {
-      return ConfigurableComponent::getProperty(name, value);
+      return ConfigurableComponent::getProperty<T>(name, value);
     }
   }
   /**
@@ -102,7 +103,7 @@ class ProcessorNode : public ConfigurableComponent, public Connectable {
    * Returns theflow version
    * @returns flow version. can be null if a flow version is not tracked.
    */
-  virtual std::shared_ptr<state::FlowIdentifier> getFlowIdentifier() const{
+  virtual std::shared_ptr<state::FlowIdentifier> getFlowIdentifier() const {
     if (processor_ != nullptr) {
       return processor_->getFlowIdentifier();
     } else {
@@ -248,9 +249,9 @@ class ProcessorNode : public ConfigurableComponent, public Connectable {
   }
   /*
 
-  unsigned const char *getUUID() {
-    return processor_->getUUID();
-  }*/
+   unsigned const char *getUUID() {
+   return processor_->getUUID();
+   }*/
   /**
    * Return the UUID string
    * @param constant reference to the UUID str
diff --git a/libminifi/include/core/Property.h b/libminifi/include/core/Property.h
index ff1a5360..ef6d253b 100644
--- a/libminifi/include/core/Property.h
+++ b/libminifi/include/core/Property.h
@@ -20,7 +20,9 @@
 
 #include <algorithm>
 #include "core/Core.h"
+#include "PropertyValidation.h"
 #include <sstream>
+#include <typeindex>
 #include <string>
 #include <vector>
 #include <queue>
@@ -32,6 +34,8 @@
 #include <set>
 #include <stdlib.h>
 #include <math.h>
+
+#include "PropertyValue.h"
 #include "utils/StringUtils.h"
 
 namespace org {
@@ -40,15 +44,6 @@ namespace nifi {
 namespace minifi {
 namespace core {
 
-enum TimeUnit {
-  DAY,
-  HOUR,
-  MINUTE,
-  SECOND,
-  MILLISECOND,
-  NANOSECOND
-};
-
 class PropertyBuilder;
 
 class Property {
@@ -67,7 +62,7 @@ class Property {
         exclusive_of_properties_(std::move(exclusive_of_properties)),
         is_collection_(false),
         supports_el_(false) {
-    values_.push_back(std::move(value));
+    default_value_ = coerceDefaultValue(value);
   }
 
   Property(const std::string name, const std::string description, std::string value)
@@ -76,7 +71,7 @@ class Property {
         is_required_(false),
         is_collection_(false),
         supports_el_(false) {
-    values_.push_back(std::move(value));
+    default_value_ = coerceDefaultValue(value);
   }
 
   Property(const std::string name, const std::string description)
@@ -85,35 +80,12 @@ class Property {
         is_required_(false),
         is_collection_(true),
         supports_el_(false) {
+    validator_ = StandardValidators::VALID;
   }
 
-  Property(Property &&other)
-      : name_(std::move(other.name_)),
-        description_(std::move(other.description_)),
-        is_required_(other.is_required_),
-        valid_regex_(std::move(other.valid_regex_)),
-        dependent_properties_(std::move(other.dependent_properties_)),
-        exclusive_of_properties_(std::move(other.exclusive_of_properties_)),
-        is_collection_(other.is_collection_),
-        values_(std::move(other.values_)),
-        display_name_(std::move(other.display_name_)),
-        types_(std::move(other.types_)),
-        supports_el_(other.supports_el_) {
-  }
-
-  Property(const Property &other)
-      : name_(other.name_),
-        description_(other.description_),
-        is_required_(other.is_required_),
-        valid_regex_(other.valid_regex_),
-        dependent_properties_(other.dependent_properties_),
-        exclusive_of_properties_(other.exclusive_of_properties_),
-        is_collection_(other.is_collection_),
-        values_(other.values_),
-        display_name_(other.display_name_),
-        types_(other.types_),
-        supports_el_(other.supports_el_) {
-  }
+  Property(Property &&other) = default;
+
+  Property(const Property &other) = default;
 
   Property()
       : name_(""),
@@ -121,6 +93,7 @@ class Property {
         is_required_(false),
         is_collection_(false),
         supports_el_(false) {
+    validator_ = StandardValidators::VALID;
   }
 
   virtual ~Property() = default;
@@ -129,21 +102,72 @@ class Property {
   std::string getDisplayName() const;
   std::vector<std::string> getAllowedTypes() const;
   std::string getDescription() const;
-  std::string getValue() const;
+  std::shared_ptr<PropertyValidator> getValidator() const;
+  const PropertyValue &getValue() const;
   bool getRequired() const;
   bool supportsExpressionLangauge() const;
   std::string getValidRegex() const;
   std::vector<std::string> getDependentProperties() const;
   std::vector<std::pair<std::string, std::string>> getExclusiveOfProperties() const;
-  std::vector<std::string> &getValues();
+  std::vector<std::string> getValues();
+
+  const PropertyValue &getDefaultValue() const {
+    return default_value_;
+  }
+
+  template<typename T = std::string>
+  void setValue(const T &value) {
+    PropertyValue vn = default_value_;
+    vn = value;
+    if (validator_) {
+      vn.setValidator(validator_);
+      ValidationResult result = validator_->validate(name_, vn.getValue());
+      if (!result.valid()) {
+        // throw some exception?
+      }
+    } else {
+      vn.setValidator(core::StandardValidators::VALID);
+    }
+    if (!is_collection_) {
+      values_.clear();
+      values_.push_back(vn);
+    } else {
+      values_.push_back(vn);
+    }
+  }
 
-  void setValue(std::string value);
+  void setValue(PropertyValue &vn) {
+    if (validator_) {
+      vn.setValidator(validator_);
+      ValidationResult result = validator_->validate(name_, vn.getValue());
+      if (!result.valid()) {
+        // throw some exception?
+      }
+    } else {
+      vn.setValidator(core::StandardValidators::VALID);
+    }
+    if (!is_collection_) {
+      values_.clear();
+      values_.push_back(vn);
+    } else {
+      values_.push_back(vn);
+    }
+  }
   void setSupportsExpressionLanguage(bool supportEl);
+
+  std::vector<PropertyValue> getAllowedValues() const {
+    return allowed_values_;
+  }
+
+  void addAllowedValue(const PropertyValue &value) {
+    allowed_values_.push_back(value);
+  }
   /**
    * Add value to the collection of values.
    */
-  void addValue(std::string value);
-  const Property &operator=(const Property &other);
+  void addValue(const std::string &value);
+  Property &operator=(const Property &other) = default;
+  Property &operator=(Property &&other) = default;
 // Compare
   bool operator <(const Property & right) const;
 
@@ -322,72 +346,10 @@ class Property {
       return false;
   }
 
-// Convert String to Integer
+  // Convert String to Integer
   template<typename T>
   static bool StringToInt(std::string input, T &output) {
-    if (input.size() == 0) {
-      return false;
-    }
-
-    const char *cvalue = input.c_str();
-    char *pEnd;
-    auto ival = std::strtoll(cvalue, &pEnd, 0);
-
-    if (pEnd[0] == '\0') {
-      output = ival;
-      return true;
-    }
-
-    while (*pEnd == ' ') {
-      // Skip the space
-      pEnd++;
-    }
-
-    char end0 = toupper(pEnd[0]);
-    if (end0 == 'B') {
-      output = ival;
-      return true;
-    } else if ((end0 == 'K') || (end0 == 'M') || (end0 == 'G') || (end0 == 'T') || (end0 == 'P')) {
-      if (pEnd[1] == '\0') {
-        unsigned long int multiplier = 1000;
-
-        if ((end0 != 'K')) {
-          multiplier *= 1000;
-          if (end0 != 'M') {
-            multiplier *= 1000;
-            if (end0 != 'G') {
-              multiplier *= 1000;
-              if (end0 != 'T') {
-                multiplier *= 1000;
-              }
-            }
-          }
-        }
-        output = ival * multiplier;
-        return true;
-
-      } else if ((pEnd[1] == 'b' || pEnd[1] == 'B') && (pEnd[2] == '\0')) {
-
-        unsigned long int multiplier = 1024;
-
-        if ((end0 != 'K')) {
-          multiplier *= 1024;
-          if (end0 != 'M') {
-            multiplier *= 1024;
-            if (end0 != 'G') {
-              multiplier *= 1024;
-              if (end0 != 'T') {
-                multiplier *= 1024;
-              }
-            }
-          }
-        }
-        output = ival * multiplier;
-        return true;
-      }
-    }
-
-    return false;
+    return DataSizeValue::StringToInt<T>(input, output);
   }
 
   static bool StringToInt(std::string input, int64_t &output) {
@@ -410,6 +372,23 @@ class Property {
 
  protected:
 
+  /**
+   * Coerce default values at construction.
+   */
+  PropertyValue coerceDefaultValue(const std::string &value) {
+    PropertyValue ret;
+    if (value == "false" || value == "true") {
+      bool val;
+      std::istringstream(value) >> std::boolalpha >> val;
+      ret = val;
+      validator_ = StandardValidators::getValidator(ret.getValue());
+    } else {
+      ret = value;
+      validator_ = StandardValidators::VALID;
+    }
+    return ret;
+  }
+
   std::string name_;
   std::string description_;
   bool is_required_;
@@ -417,8 +396,11 @@ class Property {
   std::vector<std::string> dependent_properties_;
   std::vector<std::pair<std::string, std::string>> exclusive_of_properties_;
   bool is_collection_;
-  std::vector<std::string> values_;
+  PropertyValue default_value_;
+  std::vector<PropertyValue> values_;
+  std::shared_ptr<PropertyValidator> validator_;
   std::string display_name_;
+  std::vector<PropertyValue> allowed_values_;
   // types represents the allowable types for this property
   // these types should be the canonical name.
   std::vector<std::string> types_;
@@ -426,10 +408,15 @@ class Property {
  private:
 
   friend class PropertyBuilder;
+
 };
 
+template<typename T>
+class ConstrainedProperty;
+
 class PropertyBuilder : public std::enable_shared_from_this<PropertyBuilder> {
  public:
+
   static std::shared_ptr<PropertyBuilder> createProperty(const std::string &name) {
     std::shared_ptr<PropertyBuilder> builder = std::unique_ptr<PropertyBuilder>(new PropertyBuilder());
     builder->prop.name_ = name;
@@ -458,8 +445,47 @@ class PropertyBuilder : public std::enable_shared_from_this<PropertyBuilder> {
     return shared_from_this();
   }
 
+  template<typename T>
+  std::shared_ptr<PropertyBuilder> withDefaultValue(const T &df, const std::shared_ptr<PropertyValidator> &validator = nullptr) {
+    prop.default_value_ = df;
+
+    if (validator != nullptr) {
+      prop.default_value_.setValidator(validator);
+      prop.validator_ = validator;
+    } else {
+      prop.validator_ = StandardValidators::getValidator(prop.default_value_.getValue());
+      prop.default_value_.setValidator(prop.validator_);
+    }
+    // inspect the type and add a validator to this.
+    // there may be cases in which the validator is typed differently
+    // and a validator can be added for this.
+    return shared_from_this();
+  }
+
+  template<typename T>
+  std::shared_ptr<ConstrainedProperty<T>> withAllowableValue(const T& df) {
+    auto property = std::make_shared<ConstrainedProperty<T>>(shared_from_this());
+    property->withAllowableValue(df);
+    return property;
+  }
+
+  template<typename T>
+  std::shared_ptr<ConstrainedProperty<T>> withAllowableValues(const std::set<T> &df) {
+    auto property = std::make_shared<ConstrainedProperty<T>>(shared_from_this());
+    property->withAllowableValues(df);
+    return property;
+  }
+
+  template<typename T>
   std::shared_ptr<PropertyBuilder> withDefaultValue(const std::string &df) {
-    prop.values_.push_back(std::move(df));
+    prop.default_value_.operator=<T>(df);
+
+    prop.validator_ = StandardValidators::getValidator(prop.default_value_.getValue());
+    prop.default_value_.setValidator(prop.validator_);
+
+    // inspect the type and add a validator to this.
+    // there may be cases in which the validator is typed differently
+    // and a validator can be added for this.
     return shared_from_this();
   }
 
@@ -482,7 +508,80 @@ class PropertyBuilder : public std::enable_shared_from_this<PropertyBuilder> {
 
   PropertyBuilder() {
   }
+
 };
+
+template<typename T>
+class ConstrainedProperty : public std::enable_shared_from_this<ConstrainedProperty<T>> {
+ public:
+  std::shared_ptr<ConstrainedProperty<T>> withDescription(const std::string &description) {
+    builder_->withDescription(description);
+    return this->shared_from_this();
+  }
+
+  std::shared_ptr<ConstrainedProperty<T>> isRequired(bool required) {
+    builder_->isRequired(required);
+    return this->shared_from_this();
+  }
+
+  std::shared_ptr<ConstrainedProperty<T>> supportsExpressionLanguage(bool sel) {
+    builder_->supportsExpressionLanguage(sel);
+    return this->shared_from_this();
+  }
+
+  std::shared_ptr<ConstrainedProperty<T>> withDefaultValue(const T &df, const std::shared_ptr<PropertyValidator> &validator = nullptr) {
+    builder_->withDefaultValue(df, validator);
+    return this->shared_from_this();
+  }
+
+  std::shared_ptr<ConstrainedProperty<T>> withAllowableValue(const T& df) {
+    PropertyValue dn;
+    dn = df;
+    allowed_values_.emplace_back(dn);
+    return this->shared_from_this();
+  }
+
+  std::shared_ptr<ConstrainedProperty<T>> withAllowableValues(const std::set<T>& defaultValues) {
+    for (const auto &defaultValue : defaultValues) {
+      PropertyValue dn;
+      dn = defaultValue;
+      allowed_values_.emplace_back(dn);
+    }
+    return this->shared_from_this();
+  }
+
+  template<typename J>
+  std::shared_ptr<ConstrainedProperty<T>> asType() {
+    builder_->asType<J>();
+    return this->shared_from_this();
+  }
+
+  std::shared_ptr<ConstrainedProperty<T>> withExclusiveProperty(const std::string &property, const std::string regex) {
+    builder_->withExclusiveProperty(property, regex);
+    return this->shared_from_this();
+  }
+
+  Property &&build() {
+    Property &&prop = builder_->build();
+    for (const auto &value : allowed_values_) {
+      prop.addAllowedValue(value);
+    }
+    return std::move(prop);
+  }
+
+  ConstrainedProperty(const std::shared_ptr<PropertyBuilder> &builder)
+      : builder_(builder) {
+
+  }
+
+ protected:
+
+  std::vector<PropertyValue> allowed_values_;
+  std::shared_ptr<PropertyBuilder> builder_;
+
+  friend class PropertyBuilder;
+};
+
 } /* namespace core */
 } /* namespace minifi */
 } /* namespace nifi */
diff --git a/libminifi/include/core/PropertyValidation.h b/libminifi/include/core/PropertyValidation.h
new file mode 100644
index 00000000..fc2a9811
--- /dev/null
+++ b/libminifi/include/core/PropertyValidation.h
@@ -0,0 +1,347 @@
+/**
+ *
+ * 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_CORE_PROPERTYVALIDATION_H_
+#define LIBMINIFI_INCLUDE_CORE_PROPERTYVALIDATION_H_
+
+#include "core/Core.h"
+#include "core/state/Value.h"
+#include "TypedValues.h"
+#include "utils/StringUtils.h"
+#include <memory>
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+class ValidationResult;
+
+class ValidationResult {
+ public:
+  bool valid() const {
+    return valid_;
+  }
+
+  class Builder {
+   public:
+    static Builder createBuilder() {
+      return Builder();
+    }
+    Builder &isValid(bool valid) {
+      valid_ = valid;
+      return *this;
+    }
+    Builder &withSubject(const std::string &subject) {
+      subject_ = subject;
+      return *this;
+    }
+    Builder &withInput(const std::string &input) {
+      input_ = input;
+      return *this;
+    }
+
+    ValidationResult build() {
+      return ValidationResult(*this);
+    }
+
+   protected:
+    bool valid_;
+    std::string subject_;
+    std::string input_;
+    friend class ValidationResult;
+  };
+ private:
+
+  bool valid_;
+  std::string subject_;
+  std::string input_;
+
+  ValidationResult(const Builder &builder)
+      : valid_(builder.valid_),
+        subject_(builder.subject_),
+        input_(builder.input_) {
+  }
+
+  friend class Builder;
+};
+
+class PropertyValidator {
+ public:
+
+  PropertyValidator(const std::string &name)
+      : name_(name) {
+  }
+  virtual ~PropertyValidator() {
+
+  }
+
+  std::string getName() const {
+    return name_;
+  }
+
+  virtual ValidationResult validate(const std::string &subject, const std::shared_ptr<minifi::state::response::Value> &input) const = 0;
+
+  virtual ValidationResult validate(const std::string &subject, const std::string &input) const = 0;
+
+ protected:
+  template<typename T>
+  ValidationResult _validate_internal(const std::string &subject, const std::shared_ptr<minifi::state::response::Value> &input) const {
+    if (std::dynamic_pointer_cast<T>(input) != nullptr) {
+      return ValidationResult::Builder::createBuilder().withSubject(subject).withInput(input->getStringValue()).isValid(true).build();
+    } else {
+      state::response::ValueNode vn;
+      vn = input->getStringValue();
+      return validate(subject, input->getStringValue());
+    }
+
+  }
+
+  std::string name_;
+};
+
+class AlwaysValid : public PropertyValidator {
+  bool always_valid_;
+ public:
+  AlwaysValid(bool isalwaysvalid, const std::string &name)
+      : always_valid_(isalwaysvalid),
+        PropertyValidator(name) {
+
+  }
+  virtual ~AlwaysValid() {
+  }
+  ValidationResult validate(const std::string &subject, const std::shared_ptr<minifi::state::response::Value> &input) const {
+    return ValidationResult::Builder::createBuilder().withSubject(subject).withInput(input->getStringValue()).isValid(always_valid_).build();
+  }
+
+  ValidationResult validate(const std::string &subject, const std::string &input) const {
+    return ValidationResult::Builder::createBuilder().withSubject(subject).withInput(input).isValid(always_valid_).build();
+  }
+
+};
+
+class BooleanValidator : public PropertyValidator {
+ public:
+  BooleanValidator(const std::string &name)
+      : PropertyValidator(name) {
+  }
+  virtual ~BooleanValidator() {
+
+  }
+
+  ValidationResult validate(const std::string &subject, const std::shared_ptr<minifi::state::response::Value> &input) const {
+    return PropertyValidator::_validate_internal<minifi::state::response::BoolValue>(subject, input);
+  }
+
+  ValidationResult validate(const std::string &subject, const std::string &input) const {
+    if (utils::StringUtils::equalsIgnoreCase(input, "true") || utils::StringUtils::equalsIgnoreCase(input, "false"))
+      return ValidationResult::Builder::createBuilder().withSubject(subject).withInput(input).isValid(true).build();
+    else
+      return ValidationResult::Builder::createBuilder().withSubject(subject).withInput(input).isValid(false).build();
+  }
+};
+
+class IntegerValidator : public PropertyValidator {
+ public:
+  IntegerValidator(const std::string &name)
+      : PropertyValidator(name) {
+  }
+  virtual ~IntegerValidator() {
+  }
+
+  ValidationResult validate(const std::string &subject, const std::shared_ptr<minifi::state::response::Value> &input) const {
+    return PropertyValidator::_validate_internal<minifi::state::response::IntValue>(subject, input);
+  }
+
+  ValidationResult validate(const std::string &subject, const std::string &input) const {
+    try {
+      std::stoi(input);
+      return ValidationResult::Builder::createBuilder().withSubject(subject).withInput(input).isValid(true).build();
+    } catch (...) {
+
+    }
+    return ValidationResult::Builder::createBuilder().withSubject(subject).withInput(input).isValid(false).build();
+  }
+};
+
+class LongValidator : public PropertyValidator {
+ public:
+  explicit LongValidator(const std::string &name, int64_t min = (std::numeric_limits<int64_t>::min)(), int64_t max = (std::numeric_limits<int64_t>::max)())
+      : PropertyValidator(name),
+        min_(min),
+        max_(max) {
+  }
+  virtual ~LongValidator() {
+
+  }
+  ValidationResult validate(const std::string &subject, const std::shared_ptr<minifi::state::response::Value> &input) const {
+    auto in64 = std::dynamic_pointer_cast<minifi::state::response::Int64Value>(input);
+    if (in64) {
+      return ValidationResult::Builder::createBuilder().withSubject(subject).withInput(in64->getStringValue()).isValid(in64->getValue() >= min_ && in64->getValue() <= max_).build();
+    } else {
+      auto intb = std::dynamic_pointer_cast<minifi::state::response::IntValue>(input);
+      return ValidationResult::Builder::createBuilder().withSubject(subject).withInput(intb->getStringValue()).isValid(intb->getValue() >= min_ && intb->getValue() <= max_).build();
+    }
+  }
+
+  ValidationResult validate(const std::string &subject, const std::string &input) const {
+    try {
+      auto res = std::stoll(input);
+
+      return ValidationResult::Builder::createBuilder().withSubject(subject).withInput(input).isValid(res >= min_ && res <= max_).build();
+    } catch (...) {
+
+    }
+    return ValidationResult::Builder::createBuilder().withSubject(subject).withInput(input).isValid(false).build();
+  }
+
+ private:
+  int64_t min_;
+  int64_t max_;
+};
+
+class UnsignedLongValidator : public PropertyValidator {
+ public:
+  explicit UnsignedLongValidator(const std::string &name)
+      : PropertyValidator(name) {
+  }
+  virtual ~UnsignedLongValidator() {
+
+  }
+  ValidationResult validate(const std::string &subject, const std::shared_ptr<minifi::state::response::Value> &input) const {
+    return PropertyValidator::_validate_internal<minifi::state::response::UInt64Value>(subject, input);
+  }
+
+  ValidationResult validate(const std::string &subject, const std::string &input) const {
+    try {
+      std::stoull(input);
+      return ValidationResult::Builder::createBuilder().withSubject(subject).withInput(input).isValid(true).build();
+    } catch (...) {
+
+    }
+    return ValidationResult::Builder::createBuilder().withSubject(subject).withInput(input).isValid(false).build();
+  }
+
+};
+
+class DataSizeValidator : public PropertyValidator {
+ public:
+  DataSizeValidator(const std::string &name)
+      : PropertyValidator(name) {
+  }
+  virtual ~DataSizeValidator() {
+
+  }
+  ValidationResult validate(const std::string &subject, const std::shared_ptr<minifi::state::response::Value> &input) const {
+    return PropertyValidator::_validate_internal<core::DataSizeValue>(subject, input);
+  }
+
+  ValidationResult validate(const std::string &subject, const std::string &input) const {
+    uint64_t out;
+    if (core::DataSizeValue::StringToInt(input, out))
+      return ValidationResult::Builder::createBuilder().withSubject(subject).withInput(input).isValid(true).build();
+    else
+      return ValidationResult::Builder::createBuilder().withSubject(subject).withInput(input).isValid(false).build();
+  }
+};
+
+class PortValidator : public LongValidator {
+ public:
+  PortValidator(const std::string &name)
+      : LongValidator(name, 1, 65535) {
+  }
+  virtual ~PortValidator() {
+
+  }
+};
+
+class TimePeriodValidator : public PropertyValidator {
+ public:
+  TimePeriodValidator(const std::string &name)
+      : PropertyValidator(name) {
+  }
+  virtual ~TimePeriodValidator() {
+
+  }
+  ValidationResult validate(const std::string &subject, const std::shared_ptr<minifi::state::response::Value> &input) const {
+    return PropertyValidator::_validate_internal<core::TimePeriodValue>(subject, input);
+  }
+
+  ValidationResult validate(const std::string &subject, const std::string &input) const {
+    uint64_t out;
+    TimeUnit outTimeUnit;
+    if (core::TimePeriodValue::StringToTime(input, out, outTimeUnit))
+      return ValidationResult::Builder::createBuilder().withSubject(subject).withInput(input).isValid(true).build();
+    else
+      return ValidationResult::Builder::createBuilder().withSubject(subject).withInput(input).isValid(false).build();
+  }
+};
+
+// STATIC DEFINITIONS
+
+class StandardValidators {
+ public:
+
+  static std::shared_ptr<PropertyValidator> VALID;
+
+  static const std::shared_ptr<PropertyValidator> &getValidator(const std::shared_ptr<minifi::state::response::Value> &input) {
+    static StandardValidators init;
+    if (std::dynamic_pointer_cast<core::DataSizeValue>(input) != nullptr) {
+      return init.DATA_SIZE_VALIDATOR;
+    } else if (std::dynamic_pointer_cast<core::TimePeriodValue>(input) != nullptr) {
+      return init.TIME_PERIOD_VALIDATOR;
+    } else if (std::dynamic_pointer_cast<core::DataSizeValue>(input) != nullptr) {
+      return init.DATA_SIZE_VALIDATOR;
+    } else if (std::dynamic_pointer_cast<minifi::state::response::BoolValue>(input) != nullptr) {
+      return init.BOOLEAN_VALIDATOR;
+    } else if (std::dynamic_pointer_cast<minifi::state::response::IntValue>(input) != nullptr) {
+      return init.INTEGER_VALIDATOR;
+    } else if (std::dynamic_pointer_cast<minifi::state::response::Int64Value>(input) != nullptr) {
+      return init.LONG_VALIDATOR;
+    } else if (std::dynamic_pointer_cast<minifi::state::response::UInt64Value>(input) != nullptr) {
+      return init.UNSIGNED_LONG_VALIDATOR;
+    } else {
+      return init.VALID;
+    }
+  }
+
+  static const std::shared_ptr<PropertyValidator> PORT_VALIDATOR(){
+    static std::shared_ptr<PropertyValidator> validator = std::make_shared<PortValidator>("PORT_VALIDATOR");
+    return validator;
+  }
+
+ private:
+  std::shared_ptr<PropertyValidator> INVALID;
+  std::shared_ptr<PropertyValidator> INTEGER_VALIDATOR;
+  std::shared_ptr<PropertyValidator> LONG_VALIDATOR;
+  std::shared_ptr<PropertyValidator> UNSIGNED_LONG_VALIDATOR;
+  std::shared_ptr<PropertyValidator> BOOLEAN_VALIDATOR;
+  std::shared_ptr<PropertyValidator> DATA_SIZE_VALIDATOR;
+  std::shared_ptr<PropertyValidator> TIME_PERIOD_VALIDATOR;
+
+  StandardValidators();
+}
+;
+
+} /* namespace core */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
+
+#endif /* LIBMINIFI_INCLUDE_CORE_PROPERTYVALIDATION_H_ */
diff --git a/libminifi/include/core/PropertyValue.h b/libminifi/include/core/PropertyValue.h
new file mode 100644
index 00000000..468c4f77
--- /dev/null
+++ b/libminifi/include/core/PropertyValue.h
@@ -0,0 +1,218 @@
+/**
+ *
+ * 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_CORE_TYPES_PROPERTYVALUE_H_
+#define LIBMINIFI_INCLUDE_CORE_TYPES_PROPERTYVALUE_H_
+
+#include "state/Value.h"
+#include "PropertyValidation.h"
+#include <typeindex>
+#include "TypedValues.h"
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+static inline std::shared_ptr<state::response::Value> convert(const std::shared_ptr<state::response::Value> &prior, const std::string &ref) {
+  if (prior->getTypeIndex() == state::response::Value::UINT64_TYPE) {
+    // there are specializations, so check them first
+    if (std::dynamic_pointer_cast<TimePeriodValue>(prior)) {
+      return std::make_shared<TimePeriodValue>(ref);
+    } else if (std::dynamic_pointer_cast<DataSizeValue>(prior)) {
+      return std::make_shared<DataSizeValue>(ref);
+    } else {
+      return std::make_shared<state::response::UInt64Value>(ref);
+    }
+  } else if (prior->getTypeIndex() == state::response::Value::INT64_TYPE) {
+    return std::make_shared<state::response::Int64Value>(ref);
+  } else if (prior->getTypeIndex() == state::response::Value::INT_TYPE) {
+    return std::make_shared<state::response::IntValue>(ref);
+  } else if (prior->getTypeIndex() == state::response::Value::BOOL_TYPE) {
+    return std::make_shared<state::response::BoolValue>(ref);
+  } else {
+    return std::make_shared<state::response::Value>(ref);
+  }
+}
+
+/**
+ * Purpose and Design: PropertyValue extends ValueNode, bringing with it value_.
+ * The reason for this is that there are additional features to support validation
+ * and value translation.
+ */
+class PropertyValue : public state::response::ValueNode {
+ public:
+  PropertyValue()
+      : type_id(std::type_index(typeid(std::string))),
+        ValueNode() {
+    validator_ = StandardValidators::VALID;
+  }
+
+  PropertyValue(const PropertyValue &o)
+      : type_id(o.type_id),
+        validator_(o.validator_),
+        state::response::ValueNode(o) {
+  }
+  PropertyValue(PropertyValue &&o)
+      : type_id(o.type_id),
+        validator_(std::move(o.validator_)),
+        state::response::ValueNode(std::move(o)) {
+  }
+
+  void setValidator(const std::shared_ptr<PropertyValidator> &val) {
+    validator_ = val;
+  }
+
+  std::shared_ptr<PropertyValidator> getValidator() const {
+    return validator_;
+  }
+
+  ValidationResult validate(const std::string &subject) const {
+    return validator_->validate(subject, getValue());
+  }
+
+  operator uint64_t() const {
+    uint64_t res;
+    if (value_->convertValue(res)) {
+      return res;
+    }
+    throw std::runtime_error("Invalid conversion to uint64_t for" + value_->getStringValue());
+  }
+
+  operator int64_t() const {
+    int64_t res;
+    if (value_->convertValue(res)) {
+      return res;
+    }
+    throw std::runtime_error("Invalid conversion to int64_t");
+  }
+
+  operator int() const {
+    int res;
+    if (value_->convertValue(res)) {
+      return res;
+    }
+    throw std::runtime_error("Invalid conversion to int ");
+  }
+
+  operator bool() const {
+    bool res;
+    if (value_->convertValue(res)) {
+      return res;
+    }
+    throw std::runtime_error("Invalid conversion to bool");
+  }
+
+  PropertyValue &operator=(PropertyValue &&o) = default;
+  PropertyValue &operator=(const PropertyValue &o) = default;
+
+  operator std::string() const {
+    return to_string();
+  }
+
+  std::type_index getTypeInfo() const {
+    return type_id;
+  }
+  /**
+   * Define the representations and eventual storage relationships through
+   * createValue
+   */
+  template<typename T>
+  auto operator=(const T ref) -> typename std::enable_if<std::is_same<T, std::string >::value,PropertyValue&>::type {
+    if (value_ == nullptr) {
+      type_id = std::type_index(typeid(T));
+      value_ = minifi::state::response::createValue(ref);
+    } else {
+      type_id = std::type_index(typeid(T));
+      auto ret = convert(value_, ref);
+      if (ret != nullptr) {
+        value_ = ret;
+      } else {
+        /**
+         * This is a protection mechanism that allows us to fail properties that are strictly defined.
+         * To maintain backwards compatibility we allow strings to be set by way of the internal API
+         * We then rely on the developer of the processor to perform the conversion. We want to get away from
+         * this, so this exception will throw an exception, forcefully, when they specify types in properties.
+         */
+        throw std::runtime_error("Invalid conversion");
+      }
+
+    }
+    return *this;
+  }
+
+  template<typename T>
+  auto operator=(const T ref) -> typename std::enable_if<std::is_same<T, int >::value ||
+  std::is_same<T, uint32_t >::value ||
+  std::is_same<T, uint64_t >::value ||
+  std::is_same<T, int64_t >::value ||
+  std::is_same<T, bool >::value,PropertyValue&>::type {
+    if (value_ == nullptr) {
+      type_id = std::type_index(typeid(T));
+      value_ = minifi::state::response::createValue(ref);
+    } else {
+      if (std::dynamic_pointer_cast<DataSizeValue>(value_)) {
+        value_ = std::make_shared<DataSizeValue>(ref);
+        type_id = DataSizeValue::type_id;
+      } else if (std::dynamic_pointer_cast<TimePeriodValue>(value_)) {
+        value_ = std::make_shared<TimePeriodValue>(ref);
+        type_id = TimePeriodValue::type_id;
+      } else if (type_id == std::type_index(typeid(T))) {
+        value_ = minifi::state::response::createValue(ref);
+      } else {
+        // this is not the place to perform translation. There are other places within
+        // the code where we can do assignments and transformations from "10" to (int)10;
+        throw std::runtime_error("Assigning invalid types");
+      }
+    }
+    return *this;
+  }
+
+  template<typename T>
+  auto operator=(const T ref) -> typename std::enable_if<
+  std::is_same<T, char* >::value ||
+  std::is_same<T, const char* >::value,PropertyValue&>::type {
+    // translated these into strings
+    return operator=<std::string>(std::string(ref));
+  }
+
+  template<typename T>
+  auto operator=(const std::string &ref) -> typename std::enable_if<
+  std::is_same<T, DataSizeValue >::value ||
+  std::is_same<T, TimePeriodValue >::value,PropertyValue&>::type {
+    value_ = std::make_shared<T>(ref);
+    type_id = value_->getTypeIndex();
+    return *this;
+  }
+
+ protected:
+
+  std::type_index type_id;
+  std::shared_ptr<PropertyValidator> validator_;
+};
+
+inline char const* conditional_conversion(const PropertyValue &v) {
+  return v.getValue()->getStringValue().c_str();
+}
+
+} /* namespace core */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
+
+#endif /* LIBMINIFI_INCLUDE_CORE_TYPES_PROPERTYVALUE_H_ */
diff --git a/libminifi/include/core/TypedValues.h b/libminifi/include/core/TypedValues.h
new file mode 100644
index 00000000..212635ad
--- /dev/null
+++ b/libminifi/include/core/TypedValues.h
@@ -0,0 +1,232 @@
+/**
+ *
+ * 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_CORE_TYPEDVALUES_H_
+#define LIBMINIFI_INCLUDE_CORE_TYPEDVALUES_H_
+
+#include "state/Value.h"
+#include <typeindex>
+#include "utils/StringUtils.h"
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+class TransformableValue {
+ public:
+  TransformableValue() {
+  }
+};
+
+
+/**
+ * Purpose and Design: TimePeriodValue represents a time period that can be set via a numeric followed by
+ * a time unit string. This Value is based on uint64, but has the support to return
+ * the original string representation. Once set, both are immutable.
+ */
+class TimePeriodValue : public TransformableValue, public state::response::UInt64Value {
+ public:
+  static const std::type_index type_id;
+
+  explicit TimePeriodValue(const std::string &timeString)
+      : state::response::UInt64Value(0) {
+    TimeUnit units;
+    StringToTime(timeString, value, units);
+    string_value = timeString;
+    ConvertTimeUnitToMS<uint64_t>(value, units, value);
+  }
+
+  explicit TimePeriodValue(uint64_t value)
+      : state::response::UInt64Value(value) {
+  }
+
+  // Convert TimeUnit to MilliSecond
+  template<typename T>
+  static bool ConvertTimeUnitToMS(T input, TimeUnit unit, T &out) {
+    if (unit == MILLISECOND) {
+      out = input;
+      return true;
+    } else if (unit == SECOND) {
+      out = input * 1000;
+      return true;
+    } else if (unit == MINUTE) {
+      out = input * 60 * 1000;
+      return true;
+    } else if (unit == HOUR) {
+      out = input * 60 * 60 * 1000;
+      return true;
+    } else if (unit == DAY) {
+      out = 24 * 60 * 60 * 1000;
+      return true;
+    } else if (unit == NANOSECOND) {
+      out = input / 1000 / 1000;
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  static bool StringToTime(std::string input, uint64_t &output, TimeUnit &timeunit) {
+    if (input.size() == 0) {
+      return false;
+    }
+
+    const char *cvalue = input.c_str();
+    char *pEnd;
+    auto ival = std::strtoll(cvalue, &pEnd, 0);
+
+    if (pEnd[0] == '\0') {
+      return false;
+    }
+
+    while (*pEnd == ' ') {
+      // Skip the space
+      pEnd++;
+    }
+
+    std::string unit(pEnd);
+    std::transform(unit.begin(), unit.end(), unit.begin(), ::tolower);
+
+    if (unit == "sec" || unit == "s" || unit == "second" || unit == "seconds" || unit == "secs") {
+      timeunit = SECOND;
+      output = ival;
+      return true;
+    } else if (unit == "msec" || unit == "ms" || unit == "millisecond" || unit == "milliseconds" || unit == "msecs") {
+      timeunit = MILLISECOND;
+      output = ival;
+      return true;
+    } else if (unit == "min" || unit == "m" || unit == "mins" || unit == "minute" || unit == "minutes") {
+      timeunit = MINUTE;
+      output = ival;
+      return true;
+    } else if (unit == "ns" || unit == "nano" || unit == "nanos" || unit == "nanoseconds") {
+      timeunit = NANOSECOND;
+      output = ival;
+      return true;
+    } else if (unit == "ms" || unit == "milli" || unit == "millis" || unit == "milliseconds") {
+      timeunit = MILLISECOND;
+      output = ival;
+      return true;
+    } else if (unit == "h" || unit == "hr" || unit == "hour" || unit == "hrs" || unit == "hours") {
+      timeunit = HOUR;
+      output = ival;
+      return true;
+    } else if (unit == "d" || unit == "day" || unit == "days") {
+      timeunit = DAY;
+      output = ival;
+      return true;
+    } else
+      return false;
+  }
+};
+
+/**
+ * Purpose and Design: DataSizeValue represents a file system size value that extends
+ * Uint64Value. This means that a string is converted to uint64_t. The string is of the
+ * format <numeric> <byte size>.
+ */
+class DataSizeValue : public TransformableValue, public state::response::UInt64Value {
+ public:
+  static const std::type_index type_id;
+
+  explicit DataSizeValue(const std::string &sizeString)
+      : state::response::UInt64Value(0) {
+    StringToInt<uint64_t>(sizeString, value);
+    string_value = sizeString;
+  }
+
+  explicit DataSizeValue(uint64_t value)
+      : state::response::UInt64Value(value) {
+  }
+
+  // Convert String to Integer
+  template<typename T>
+  static bool StringToInt(const std::string &input, T &output) {
+    if (input.size() == 0) {
+      return false;
+    }
+
+    const char *cvalue = input.c_str();
+    char *pEnd;
+    auto ival = std::strtoll(cvalue, &pEnd, 0);
+
+    if (pEnd[0] == '\0') {
+      output = ival;
+      return true;
+    }
+
+    while (*pEnd == ' ') {
+      // Skip the space
+      pEnd++;
+    }
+
+    char end0 = toupper(pEnd[0]);
+    if (end0 == 'B') {
+      output = ival;
+      return true;
+    } else if ((end0 == 'K') || (end0 == 'M') || (end0 == 'G') || (end0 == 'T') || (end0 == 'P')) {
+      if (pEnd[1] == '\0') {
+        unsigned long int multiplier = 1000;
+
+        if ((end0 != 'K')) {
+          multiplier *= 1000;
+          if (end0 != 'M') {
+            multiplier *= 1000;
+            if (end0 != 'G') {
+              multiplier *= 1000;
+              if (end0 != 'T') {
+                multiplier *= 1000;
+              }
+            }
+          }
+        }
+        output = ival * multiplier;
+        return true;
+
+      } else if ((pEnd[1] == 'b' || pEnd[1] == 'B') && (pEnd[2] == '\0')) {
+
+        unsigned long int multiplier = 1024;
+
+        if ((end0 != 'K')) {
+          multiplier *= 1024;
+          if (end0 != 'M') {
+            multiplier *= 1024;
+            if (end0 != 'G') {
+              multiplier *= 1024;
+              if (end0 != 'T') {
+                multiplier *= 1024;
+              }
+            }
+          }
+        }
+        output = ival * multiplier;
+        return true;
+      }
+    }
+
+    return false;
+  }
+};
+
+} /* namespace core */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
+
+#endif /* LIBMINIFI_INCLUDE_CORE_TYPEDVALUES_H_ */
diff --git a/libminifi/include/core/controller/ControllerServiceMap.h b/libminifi/include/core/controller/ControllerServiceMap.h
index d2293645..7b28c287 100644
--- a/libminifi/include/core/controller/ControllerServiceMap.h
+++ b/libminifi/include/core/controller/ControllerServiceMap.h
@@ -89,6 +89,15 @@ class ControllerServiceMap {
     return true;
   }
 
+  void clear(){
+    std::lock_guard<std::mutex> lock(mutex_);
+    for(const auto &node : controller_services_list_){
+      node->disable();
+    }
+    controller_services_.clear();
+    controller_services_list_.clear();
+  }
+
   /**
    * Gets all controller services.
    * @return controller service node shared pointers.
diff --git a/libminifi/include/core/state/ProcessorController.h b/libminifi/include/core/state/ProcessorController.h
index de7c673e..1fadd277 100644
--- a/libminifi/include/core/state/ProcessorController.h
+++ b/libminifi/include/core/state/ProcessorController.h
@@ -42,9 +42,13 @@ class ProcessorController : public StateController {
 
   virtual ~ProcessorController();
 
-  std::string getComponentName() {
+  virtual std::string getComponentName() const{
     return processor_->getName();
   }
+
+  virtual std::string getComponentUUID() const{
+      return processor_->getUUIDStr();
+    }
   /**
    * Start the client
    */
diff --git a/libminifi/include/core/state/UpdateController.h b/libminifi/include/core/state/UpdateController.h
index 7cd61c45..4d70d999 100644
--- a/libminifi/include/core/state/UpdateController.h
+++ b/libminifi/include/core/state/UpdateController.h
@@ -168,7 +168,9 @@ class StateController {
 
   }
 
-  virtual std::string getComponentName() = 0;
+  virtual std::string getComponentName() const= 0;
+
+  virtual std::string getComponentUUID() const = 0;
   /**
    * Start the client
    */
diff --git a/libminifi/include/core/state/Value.h b/libminifi/include/core/state/Value.h
index a80404c2..8f714a51 100644
--- a/libminifi/include/core/state/Value.h
+++ b/libminifi/include/core/state/Value.h
@@ -18,8 +18,15 @@
 #ifndef LIBMINIFI_INCLUDE_CORE_STATE_VALUE_H_
 #define LIBMINIFI_INCLUDE_CORE_STATE_VALUE_H_
 
+#include <typeindex>
+#include <limits>
+#include <sstream>
+#include <iostream>
 #include <memory>
 #include <string>
+#include <vector>
+#include <typeinfo>
+
 namespace org {
 namespace apache {
 namespace nifi {
@@ -30,13 +37,16 @@ namespace response {
 /**
  * Purpose: Represents an AST value
  * Contains an embedded string representation to be used for a toString analog.
+ *
+ * Extensions can be more strongly typed and can be used anywhere where an abstract
+ * representation is needed.
  */
 class Value {
  public:
 
   explicit Value(const std::string &value)
-      : string_value(value) {
-
+      : string_value(value),
+        type_id(std::type_index(typeid(std::string))) {
   }
 
   virtual ~Value() {
@@ -46,13 +56,59 @@ class Value {
     return string_value;
   }
 
-  bool empty(){
+  template<typename T>
+  bool convertValue(T &ref) {
+    return convertValueImpl<typename std::common_type<T>::type>(ref);
+  }
+
+  bool empty() {
     return string_value.empty();
   }
 
+  std::type_index getTypeIndex() {
+    return type_id;
+  }
+
+  static const std::type_index UINT64_TYPE;
+  static const std::type_index INT64_TYPE;
+  static const std::type_index INT_TYPE;
+  static const std::type_index BOOL_TYPE;
+  static const std::type_index STRING_TYPE;
+
  protected:
-  std::string string_value;
 
+  template<typename T>
+  bool convertValueImpl(T &ref) {
+    return getValue(ref);
+  }
+
+  template<typename T>
+  void setTypeId() {
+    type_id = std::type_index(typeid(T));
+  }
+
+  virtual bool getValue(int &ref) {
+    ref = std::stol(string_value);
+    return true;
+  }
+
+  virtual bool getValue(int64_t &ref) {
+    ref = std::stoll(string_value);
+    return true;
+  }
+
+  virtual bool getValue(uint64_t &ref) {
+    ref = std::stoull(string_value);
+    return true;
+  }
+
+  virtual bool getValue(bool &ref) {
+    std::istringstream(string_value) >> std::boolalpha >> ref;
+    return true;
+  }
+
+  std::string string_value;
+  std::type_index type_id;
 };
 
 class IntValue : public Value {
@@ -60,12 +116,39 @@ class IntValue : public Value {
   explicit IntValue(int value)
       : Value(std::to_string(value)),
         value(value) {
+    setTypeId<int>();
+  }
+
+  explicit IntValue(const std::string &strvalue)
+      : Value(strvalue),
+        value(std::stoi(strvalue)) {
 
   }
-  int getValue() {
+  int getValue() const {
     return value;
   }
+
  protected:
+
+  virtual bool getValue(int &ref) {
+    ref = value;
+    return true;
+  }
+
+  virtual bool getValue(int64_t &ref) {
+    ref = value;
+    return true;
+  }
+
+  virtual bool getValue(uint64_t &ref) {
+    ref = value;
+    return true;
+  }
+
+  virtual bool getValue(bool &ref) {
+    return false;
+  }
+
   int value;
 };
 
@@ -74,66 +157,186 @@ class BoolValue : public Value {
   explicit BoolValue(bool value)
       : Value(value ? "true" : "false"),
         value(value) {
+    setTypeId<bool>();
+  }
 
+  explicit BoolValue(const std::string &strvalue)
+      : Value(strvalue) {
+    bool l;
+    std::istringstream(strvalue) >> std::boolalpha >> l;
+    value = l;  // avoid warnings
   }
-  bool getValue() {
+
+  bool getValue() const {
     return value;
   }
  protected:
+
+  virtual bool getValue(int &ref) {
+    if (ref == 1) {
+      ref = true;
+      return true;
+    } else if (ref == 0) {
+      ref = false;
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  virtual bool getValue(int64_t &ref) {
+    if (ref == 1) {
+      ref = true;
+      return true;
+    } else if (ref == 0) {
+      ref = false;
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  virtual bool getValue(uint64_t &ref) {
+    if (ref == 1) {
+      ref = true;
+      return true;
+    } else if (ref == 0) {
+      ref = false;
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  virtual bool getValue(bool &ref) {
+    ref = value;
+    return true;
+  }
+
   bool value;
 };
 
-class Int64Value : public Value {
+class UInt64Value : public Value {
  public:
-  explicit Int64Value(uint64_t value)
+  explicit UInt64Value(uint64_t value)
       : Value(std::to_string(value)),
         value(value) {
+    setTypeId<uint64_t>();
+  }
 
+  explicit UInt64Value(const std::string &strvalue)
+      : Value(strvalue),
+        value(std::stoull(strvalue)) {
+    setTypeId<uint64_t>();
   }
-  uint64_t getValue() {
+
+  uint64_t getValue() const {
     return value;
   }
  protected:
+
+  virtual bool getValue(int &ref) {
+    return false;
+  }
+
+  virtual bool getValue(int64_t &ref) {
+    if (value <= (std::numeric_limits<int64_t>::max)()) {
+      ref = value;
+      return true;
+    }
+    return false;
+  }
+
+  virtual bool getValue(uint64_t &ref) {
+    ref = value;
+    return true;
+  }
+
+  virtual bool getValue(bool &ref) {
+    return false;
+  }
+
   uint64_t value;
 };
 
+class Int64Value : public Value {
+ public:
+  explicit Int64Value(int64_t value)
+      : Value(std::to_string(value)),
+        value(value) {
+    setTypeId<int64_t>();
+  }
+  explicit Int64Value(const std::string &strvalue)
+      : Value(strvalue),
+        value(std::stoll(strvalue)) {
+    setTypeId<int64_t>();
+  }
+
+  int64_t getValue() {
+    return value;
+  }
+ protected:
 
-static inline std::shared_ptr<Value> createValue(
-    const bool &object) {
+  virtual bool getValue(int &ref) {
+    return false;
+  }
+
+  virtual bool getValue(int64_t &ref) {
+    ref = value;
+    return true;
+  }
+
+  virtual bool getValue(uint64_t &ref) {
+    if (value >= 0) {
+      ref = value;
+      return true;
+    }
+    return false;
+  }
+
+  virtual bool getValue(bool &ref) {
+    return false;
+  }
+
+  int64_t value;
+};
+
+static inline std::shared_ptr<Value> createValue(const bool &object) {
   return std::make_shared<BoolValue>(object);
 }
 
-static inline std::shared_ptr<Value> createValue(
-    const char *object) {
+static inline std::shared_ptr<Value> createValue(const char *object) {
   return std::make_shared<Value>(object);
 }
 
-static inline std::shared_ptr<Value> createValue(
-    char *object) {
+static inline std::shared_ptr<Value> createValue(char *object) {
   return std::make_shared<Value>(std::string(object));
 }
 
-static inline std::shared_ptr<Value> createValue(
-    const std::string &object) {
+static inline std::shared_ptr<Value> createValue(const std::string &object) {
   return std::make_shared<Value>(object);
 }
 
-
-static inline std::shared_ptr<Value> createValue(
-    const uint32_t &object) {
-  return std::make_shared<Int64Value>(object);
+static inline std::shared_ptr<Value> createValue(const uint32_t &object) {
+  return std::make_shared<UInt64Value>(object);
+}
+#if ( defined(__APPLE__) || defined(__MACH__) || defined(DARWIN) )
+static inline std::shared_ptr<Value> createValue(const size_t &object) {
+  return std::make_shared<UInt64Value>(object);
 }
-static inline std::shared_ptr<Value> createValue(
-    const uint64_t &object) {
+#endif
+static inline std::shared_ptr<Value> createValue(const uint64_t &object) {
+  return std::make_shared<UInt64Value>(object);
+}
+
+static inline std::shared_ptr<Value> createValue(const int64_t &object) {
   return std::make_shared<Int64Value>(object);
 }
 
-static inline std::shared_ptr<Value> createValue(
-    const int &object) {
+static inline std::shared_ptr<Value> createValue(const int &object) {
   return std::make_shared<IntValue>(object);
 }
 
-
 /**
  * Purpose: ValueNode is the AST container for a value
  */
@@ -141,27 +344,28 @@ class ValueNode {
  public:
   ValueNode()
       : value_(nullptr) {
-
   }
 
+  ValueNode(ValueNode &&vn) = default;
+  ValueNode(const ValueNode &vn) = default;
+
   /**
    * Define the representations and eventual storage relationships through
    * createValue
    */
   template<typename T>
-  auto operator=(
-      const T ref) -> typename std::enable_if<std::is_same<T, int >::value ||
-      std::is_same<T, uint32_t >::value ||
-      std::is_same<T, uint64_t >::value ||
-      std::is_same<T, bool >::value ||
-      std::is_same<T, char* >::value ||
-      std::is_same<T, const char* >::value ||
-      std::is_same<T, std::string>::value,ValueNode&>::type {
+  auto operator=(const T ref) -> typename std::enable_if<std::is_same<T, int >::value ||
+  std::is_same<T, uint32_t >::value ||
+  std::is_same<T, size_t >::value ||
+  std::is_same<T, uint64_t >::value ||
+  std::is_same<T, bool >::value ||
+  std::is_same<T, char* >::value ||
+  std::is_same<T, const char* >::value ||
+  std::is_same<T, std::string>::value,ValueNode&>::type {
     value_ = createValue(ref);
     return *this;
   }
 
-
   ValueNode &operator=(const ValueNode &ref) {
     value_ = ref.value_;
     return *this;
@@ -199,18 +403,17 @@ struct SerializedResponseNode {
   std::string name;
   ValueNode value;
   bool array;
+  bool collapsible;
+  std::vector<SerializedResponseNode> children;
 
-  SerializedResponseNode() : array(false){
+  SerializedResponseNode(bool collapsible = true)
+      : array(false),
+        collapsible(collapsible) {
   }
 
-  std::vector<SerializedResponseNode> children;
-  SerializedResponseNode &operator=(const SerializedResponseNode &other) {
-    name = other.name;
-    value = other.value;
-    children = other.children;
-    array = other.array;
-    return *this;
-  }
+  SerializedResponseNode(const SerializedResponseNode &other) = default;
+
+  SerializedResponseNode &operator=(const SerializedResponseNode &other) = default;
 };
 
 } /* namespace metrics */
diff --git a/libminifi/include/core/state/nodes/AgentInformation.h b/libminifi/include/core/state/nodes/AgentInformation.h
index 3e08accc..0b4c30cf 100644
--- a/libminifi/include/core/state/nodes/AgentInformation.h
+++ b/libminifi/include/core/state/nodes/AgentInformation.h
@@ -122,6 +122,14 @@ class ComponentManifest : public DeviceInformation {
             descriptorDescription.name = "description";
             descriptorDescription.value = prop.second.getDescription();
 
+            SerializedResponseNode validatorName;
+            validatorName.name = "validator";
+            if (prop.second.getValidator()) {
+              validatorName.value = prop.second.getValidator()->getName();
+            } else {
+              validatorName.value = "VALID";
+            }
+
             SerializedResponseNode supportsExpressionLanguageScope;
             supportsExpressionLanguageScope.name = "expressionLanguageScope";
             supportsExpressionLanguageScope.value = prop.second.supportsExpressionLangauge() ? "FLOWFILE_ATTRIBUTES" : "NONE";
@@ -193,6 +201,7 @@ class ComponentManifest : public DeviceInformation {
               child.children.push_back(displayName);
             }
             child.children.push_back(descriptorDescription);
+            child.children.push_back(validatorName);
             child.children.push_back(descriptorRequired);
             child.children.push_back(supportsExpressionLanguageScope);
             child.children.push_back(descriptorDefaultValue);
@@ -200,6 +209,30 @@ class ComponentManifest : public DeviceInformation {
             child.children.push_back(descriptorDependentProperties);
             child.children.push_back(descriptorExclusiveOfProperties);
 
+            if (!prop.second.getAllowedValues().empty()) {
+              SerializedResponseNode allowedValues;
+              allowedValues.name = "allowableValues";
+              allowedValues.array = true;
+              for (const auto &av : prop.second.getAllowedValues()) {
+                SerializedResponseNode allowableValue;
+                allowableValue.name = "allowableValues";
+
+                SerializedResponseNode allowedValue;
+                allowedValue.name = "value";
+                allowedValue.value = av;
+                SerializedResponseNode allowedDisplayName;
+                allowedDisplayName.name = "displayName";
+                allowedDisplayName.value = av;
+
+                allowableValue.children.push_back(allowedValue);
+                allowableValue.children.push_back(allowedDisplayName);
+
+                allowedValues.children.push_back(allowableValue);
+              }
+              child.children.push_back(allowedValues);
+
+            }
+
             props.children.push_back(child);
           }
 
@@ -357,7 +390,7 @@ class AgentStatus : public StateMonitorNode {
 
       for (auto &repo : repositories_) {
         SerializedResponseNode repoNode;
-
+        repoNode.collapsible = false;
         repoNode.name = repo.first;
 
         SerializedResponseNode queuesize;
@@ -375,26 +408,28 @@ class AgentStatus : public StateMonitorNode {
     serialized.push_back(uptime);
 
     if (nullptr != monitor_) {
-      auto components = monitor_->getAllComponents();
-      SerializedResponseNode componentsNode;
-
-      componentsNode.name = "components";
-
-      for (auto component : components) {
-        SerializedResponseNode componentNode;
-
-        componentNode.name = component->getComponentName();
-
-        SerializedResponseNode componentStatusNode;
-        componentStatusNode.name = "running";
-        componentStatusNode.value = component->isRunning();
-
-        componentNode.children.push_back(componentStatusNode);
-
-        componentsNode.children.push_back(componentNode);
-      }
-      serialized.push_back(componentsNode);
-    }
+         auto components = monitor_->getAllComponents();
+         SerializedResponseNode componentsNode(false);
+         componentsNode.name = "components";
+
+         for (auto component : components) {
+           SerializedResponseNode componentNode(false);
+           componentNode.name = component->getComponentName();
+
+           SerializedResponseNode uuidNode;
+           uuidNode.name = "uuid";
+           uuidNode.value = component->getComponentUUID();
+
+           SerializedResponseNode componentStatusNode;
+           componentStatusNode.name = "running";
+           componentStatusNode.value = component->isRunning();
+
+           componentNode.children.push_back(componentStatusNode);
+           componentNode.children.push_back(uuidNode);
+           componentsNode.children.push_back(componentNode);
+         }
+         serialized.push_back(componentsNode);
+       }
 
     return serialized;
   }
diff --git a/libminifi/include/core/state/nodes/DeviceInformation.h b/libminifi/include/core/state/nodes/DeviceInformation.h
index a41cee95..59f38883 100644
--- a/libminifi/include/core/state/nodes/DeviceInformation.h
+++ b/libminifi/include/core/state/nodes/DeviceInformation.h
@@ -367,7 +367,7 @@ class DeviceInfoNode : public DeviceInformation {
 	  vcores.name = "vCores";
 	  size_t ncpus = std::thread::hardware_concurrency();
 
-	  vcores.value = std::to_string(ncpus);
+	  vcores.value = ncpus;
 
 	  systemInfo.children.push_back(vcores);
 #if defined(_SC_PHYS_PAGES) && defined(_SC_PAGESIZE)
diff --git a/libminifi/include/core/state/nodes/FlowInformation.h b/libminifi/include/core/state/nodes/FlowInformation.h
index a1a2e890..28c944ac 100644
--- a/libminifi/include/core/state/nodes/FlowInformation.h
+++ b/libminifi/include/core/state/nodes/FlowInformation.h
@@ -138,7 +138,7 @@ class FlowMonitor : public StateMonitorNode {
 
   void addConnection(const std::shared_ptr<minifi::Connection> &connection) {
     if (nullptr != connection) {
-      connections_.insert(std::make_pair(connection->getName(), connection));
+      connections_.insert(std::make_pair(connection->getUUIDStr(), connection));
     }
   }
 
@@ -186,13 +186,16 @@ class FlowInformation : public FlowMonitor {
     serialized.push_back(uri);
 
     if (!connections_.empty()) {
-      SerializedResponseNode queues;
-
+      SerializedResponseNode queues(false);
       queues.name = "queues";
 
       for (auto &queue : connections_) {
-        SerializedResponseNode repoNode;
-        repoNode.name = queue.first;
+        SerializedResponseNode repoNode(false);
+        repoNode.name = queue.second->getName();
+
+        SerializedResponseNode queueUUIDNode;
+        queueUUIDNode.name = "uuid";
+        queueUUIDNode.value = queue.second->getUUIDStr();
 
         SerializedResponseNode queuesize;
         queuesize.name = "size";
@@ -214,6 +217,7 @@ class FlowInformation : public FlowMonitor {
         repoNode.children.push_back(queuesizemax);
         repoNode.children.push_back(datasize);
         repoNode.children.push_back(datasizemax);
+        repoNode.children.push_back(queueUUIDNode);
 
         queues.children.push_back(repoNode);
 
@@ -223,21 +227,23 @@ class FlowInformation : public FlowMonitor {
 
     if (nullptr != monitor_) {
       auto components = monitor_->getAllComponents();
-      SerializedResponseNode componentsNode;
-
+      SerializedResponseNode componentsNode(false);
       componentsNode.name = "components";
 
       for (auto component : components) {
-        SerializedResponseNode componentNode;
-
+        SerializedResponseNode componentNode(false);
         componentNode.name = component->getComponentName();
 
+        SerializedResponseNode uuidNode;
+        uuidNode.name = "uuid";
+        uuidNode.value = component->getComponentUUID();
+
         SerializedResponseNode componentStatusNode;
         componentStatusNode.name = "running";
         componentStatusNode.value = component->isRunning();
 
         componentNode.children.push_back(componentStatusNode);
-
+        componentNode.children.push_back(uuidNode);
         componentsNode.children.push_back(componentNode);
       }
       serialized.push_back(componentsNode);
diff --git a/libminifi/include/core/state/nodes/MetricsBase.h b/libminifi/include/core/state/nodes/MetricsBase.h
index f4d585ee..40a0c0c8 100644
--- a/libminifi/include/core/state/nodes/MetricsBase.h
+++ b/libminifi/include/core/state/nodes/MetricsBase.h
@@ -21,9 +21,10 @@
 #include <vector>
 #include <memory>
 #include <string>
+
+#include "../Value.h"
 #include "core/Core.h"
 #include "core/Connectable.h"
-#include "core/state/Value.h"
 
 namespace org {
 namespace apache {
diff --git a/libminifi/include/core/yaml/YamlConfiguration.h b/libminifi/include/core/yaml/YamlConfiguration.h
index a8c43b8c..cf8209ea 100644
--- a/libminifi/include/core/yaml/YamlConfiguration.h
+++ b/libminifi/include/core/yaml/YamlConfiguration.h
@@ -28,6 +28,7 @@
 #include "io/StreamFactory.h"
 #include "core/logging/LoggerConfiguration.h"
 #include "utils/Id.h"
+#include "utils/StringUtils.h"
 
 namespace org {
 namespace apache {
diff --git a/libminifi/include/io/NetworkPrioritizer.h b/libminifi/include/io/NetworkPrioritizer.h
index 6c1f27f2..7f9fcad3 100644
--- a/libminifi/include/io/NetworkPrioritizer.h
+++ b/libminifi/include/io/NetworkPrioritizer.h
@@ -105,6 +105,10 @@ class NetworkPrioritizerFactory {
     return fa;
   }
 
+  void clearPrioritizer(){
+    np_ = nullptr;
+  }
+
   int setPrioritizer(const std::shared_ptr<NetworkPrioritizer> &prioritizer) {
     if (np_ != nullptr)
       return -1;
diff --git a/libminifi/include/io/validation.h b/libminifi/include/io/validation.h
index 40dd2853..85f48717 100644
--- a/libminifi/include/io/validation.h
+++ b/libminifi/include/io/validation.h
@@ -27,6 +27,21 @@
  * A checker that will, at compile time, tell us
  * if the declared type has a size method.
  */
+template<typename T>
+class empty_function_functor_checker {
+  typedef char hasit;
+  typedef long doesnothaveit;
+
+  // look for the declared type
+  template<typename O> static hasit test(decltype(&O::empty));
+  template<typename O> static doesnothaveit test(...);
+
+ public:
+  enum {
+    has_empty_function = sizeof(test<T>(0)) == sizeof(char)
+  };
+};
+
 template<typename T>
 class size_function_functor_checker {
   typedef char hasit;
@@ -51,7 +66,7 @@ static auto IsNullOrEmpty(std::string object) {
 }
 
 /**
-* Determines if the variable is null or ::size() == 0
+* Determines if the variable is null or ::empty()
 */
 template<typename T>
 static auto IsNullOrEmpty(T *object) {
@@ -60,7 +75,7 @@ static auto IsNullOrEmpty(T *object) {
 
 
 /**
-* Determines if the variable is null or ::size() == 0
+* Determines if the variable is null or ::empty()
 */
 template<typename T>
 static auto IsNullOrEmpty(std::shared_ptr<T> object){
@@ -71,37 +86,51 @@ static auto IsNullOrEmpty(std::shared_ptr<T> object){
 #else
 
 /**
-* Determines if the variable is null or ::size() == 0
+* Determines if the variable is null or ::empty()
 */
 template<typename T>
-static auto IsNullOrEmpty(T &object) -> typename std::enable_if<size_function_functor_checker<T>::has_size_function == 1, bool>::type {
-	return object.size() == 0;
+static auto IsNullOrEmpty(T &object) -> typename std::enable_if<empty_function_functor_checker<T>::has_empty_function == 1, bool>::type {
+	return object.empty();
 }
+/**
+ * Determines if the variable is null or ::empty()
+ */
+template<typename T>
+static auto IsNullOrEmpty(T *object) -> typename std::enable_if<empty_function_functor_checker<T>::has_empty_function==1, bool>::type {
+  return (nullptr == object || object->empty());
+}
+
 /**
  * Determines if the variable is null or ::size() == 0
  */
 template<typename T>
-static auto IsNullOrEmpty(T *object) -> typename std::enable_if<size_function_functor_checker<T>::has_size_function==1, bool>::type {
+static auto IsNullOrEmpty(T *object) -> typename std::enable_if<not empty_function_functor_checker<T>::has_empty_function && size_function_functor_checker<T>::has_size_function==1 , bool>::type {
   return (nullptr == object || object->size() == 0);
 }
 
+
 /**
- * Determines if the variable is null or ::size() == 0
+ * Determines if the variable is null
  */
 template<typename T>
-static auto IsNullOrEmpty(T *object) -> typename std::enable_if<not size_function_functor_checker<T>::has_size_function , bool>::type {
+static auto IsNullOrEmpty(T *object) -> typename std::enable_if<not empty_function_functor_checker<T>::has_empty_function && not size_function_functor_checker<T>::has_size_function , bool>::type {
   return (nullptr == object);
 }
 
-
 /**
-* Determines if the variable is null or ::size() == 0
+* Determines if the variable is null or ::empty()
 */
 template<typename T>
-static auto IsNullOrEmpty(std::shared_ptr<T> object) -> typename std::enable_if<not size_function_functor_checker<T>::has_size_function, bool>::type {
+static auto IsNullOrEmpty(std::shared_ptr<T> object) -> typename std::enable_if<not empty_function_functor_checker<T>::has_empty_function, bool>::type {
 	return (nullptr == object || nullptr == object.get());
 }
 
+template<typename T>
+static auto IsNullOrEmpty(std::shared_ptr<T> object) -> typename std::enable_if<not empty_function_functor_checker<T>::has_empty_function && size_function_functor_checker<T>::has_size_function==1, bool>::type {
+  return (nullptr == object || nullptr == object.get() || object->size() == 0);
+}
+
+
 #endif
 
 #endif
diff --git a/libminifi/include/processors/GenerateFlowFile.h b/libminifi/include/processors/GenerateFlowFile.h
index e7afc8f3..019aca84 100644
--- a/libminifi/include/processors/GenerateFlowFile.h
+++ b/libminifi/include/processors/GenerateFlowFile.h
@@ -39,7 +39,7 @@ class GenerateFlowFile : public core::Processor {
    * Create a new processor
    */
   GenerateFlowFile(std::string name, utils::Identifier uuid = utils::Identifier())
-      : Processor(name, uuid) {
+      : Processor(name, uuid), logger_(logging::LoggerFactory<GenerateFlowFile>::getLogger()) {
     _data = NULL;
     _dataSize = 0;
   }
@@ -89,6 +89,8 @@ class GenerateFlowFile : public core::Processor {
   char * _data;
   // Size of the generated data
   uint64_t _dataSize;
+  // logger instance
+  std::shared_ptr<logging::Logger> logger_;
 };
 
 REGISTER_RESOURCE(GenerateFlowFile, "This processor creates FlowFiles with random data or custom content. GenerateFlowFile is useful for load testing, configuration, and simulation.");
diff --git a/libminifi/include/processors/GetTCP.h b/libminifi/include/processors/GetTCP.h
index 79d76e82..7b9e0a5a 100644
--- a/libminifi/include/processors/GetTCP.h
+++ b/libminifi/include/processors/GetTCP.h
@@ -265,9 +265,9 @@ class GetTCP : public core::Processor, public state::response::MetricsNodeSource
 
   int8_t endOfMessageByte;
 
-  int64_t reconnect_interval_;
+  uint64_t reconnect_interval_;
 
-  int64_t receive_buffer_size_;
+  uint64_t receive_buffer_size_;
 
   uint16_t connection_attempt_limit_;
 
diff --git a/libminifi/include/properties/Configure.h b/libminifi/include/properties/Configure.h
index 4fb68dc5..7fda3dcb 100644
--- a/libminifi/include/properties/Configure.h
+++ b/libminifi/include/properties/Configure.h
@@ -42,6 +42,7 @@ class Configure : public Properties {
   // nifi.flow.configuration.file
   static const char *nifi_default_directory;
   static const char *nifi_flow_configuration_file;
+  static const char *nifi_flow_configuration_file_exit_failure;
   static const char *nifi_flow_configuration_file_backup_update;
   static const char *nifi_flow_engine_threads;
   static const char *nifi_administrative_yield_duration;
@@ -78,9 +79,10 @@ class Configure : public Properties {
   static const char *nifi_rest_api_user_name;
   static const char *nifi_rest_api_password;
   // c2 options
-
   static const char *nifi_c2_enable;
   static const char *nifi_c2_file_watch;
+  static const char *nifi_c2_flow_id;
+  static const char *nifi_c2_flow_url;
 
  private:
   std::string agent_identifier_;
diff --git a/libminifi/include/properties/Properties.h b/libminifi/include/properties/Properties.h
index eadb77d9..7ece1ee5 100644
--- a/libminifi/include/properties/Properties.h
+++ b/libminifi/include/properties/Properties.h
@@ -48,9 +48,10 @@ class Properties {
     properties_.clear();
   }
   // Set the config value
-  void set(std::string key, std::string value) {
+  void set(const std::string &key, const std::string &value) {
     std::lock_guard<std::mutex> lock(mutex_);
     properties_[key] = value;
+    dirty_ = true;
   }
   // Check whether the config value existed
   bool has(std::string key) {
@@ -83,7 +84,7 @@ class Properties {
   int getInt(const std::string &key, int default_value);
 
   // Parse one line in configure file like key=value
-  void parseConfigureFileLine(char *buf);
+  bool parseConfigureFileLine(char *buf, std::string &prop_key, std::string &prop_value);
   // Load Configure File
   void loadConfigureFile(const char *fileName);
   // Set the determined MINIFI_HOME
@@ -106,10 +107,21 @@ class Properties {
   // Parse Command Line
   void parseCommandLine(int argc, char **argv);
 
+  bool persistProperties();
+
  protected:
+
+  bool validateConfigurationFile(const std::string &file);
+
   std::map<std::string, std::string> properties_;
 
+
  private:
+
+  std::atomic<bool> dirty_;
+
+  std::string properties_file_;
+
   // Mutex for protection
   std::mutex mutex_;
   // Logger
diff --git a/libminifi/include/utils/StringUtils.h b/libminifi/include/utils/StringUtils.h
index 1e0c1085..2158b47e 100644
--- a/libminifi/include/utils/StringUtils.h
+++ b/libminifi/include/utils/StringUtils.h
@@ -28,6 +28,15 @@
 #include <map>
 #include "utils/FailurePolicy.h"
 
+enum TimeUnit {
+  DAY,
+  HOUR,
+  MINUTE,
+  SECOND,
+  MILLISECOND,
+  NANOSECOND
+};
+
 namespace org {
 namespace apache {
 namespace nifi {
@@ -252,6 +261,19 @@ class StringUtils {
 };
 
 } /* namespace utils */
+
+namespace core{
+enum TimeUnit {
+  DAY,
+  HOUR,
+  MINUTE,
+  SECOND,
+  MILLISECOND,
+  NANOSECOND
+};
+
+} /* namespace core */
+
 } /* namespace minifi */
 } /* namespace nifi */
 } /* namespace apache */
diff --git a/libminifi/include/utils/ThreadPool.h b/libminifi/include/utils/ThreadPool.h
index fc54a25f..5bbd3f63 100644
--- a/libminifi/include/utils/ThreadPool.h
+++ b/libminifi/include/utils/ThreadPool.h
@@ -638,6 +638,10 @@ void ThreadPool<T>::shutdown() {
       manager_thread_.join();
     {
       std::unique_lock<std::mutex> lock(worker_queue_mutex_);
+      for(const auto &thread : thread_queue_){
+        if (thread->thread_.joinable())
+        thread->thread_.join();
+      }
       thread_queue_.clear();
       current_workers_ = 0;
       while (worker_queue_.size_approx() > 0) {
diff --git a/libminifi/src/Configure.cpp b/libminifi/src/Configure.cpp
index 4bcf3151..58c64542 100644
--- a/libminifi/src/Configure.cpp
+++ b/libminifi/src/Configure.cpp
@@ -25,6 +25,7 @@ namespace minifi {
 const char *Configure::nifi_default_directory = "nifi.default.directory";
 const char *Configure::nifi_c2_enable = "nifi.c2.enable";
 const char *Configure::nifi_flow_configuration_file = "nifi.flow.configuration.file";
+const char *Configure::nifi_flow_configuration_file_exit_failure = "nifi.flow.configuration.file.exit.onfailure";
 const char *Configure::nifi_flow_configuration_file_backup_update = "nifi.flow.configuration.backup.on.update";
 const char *Configure::nifi_flow_engine_threads = "nifi.flow.engine.threads";
 const char *Configure::nifi_administrative_yield_duration = "nifi.administrative.yield.duration";
@@ -56,6 +57,8 @@ const char *Configure::nifi_security_client_ca_certificate = "nifi.security.clie
 const char *Configure::nifi_rest_api_user_name = "nifi.rest.api.user.name";
 const char *Configure::nifi_rest_api_password = "nifi.rest.api.password";
 const char *Configure::nifi_c2_file_watch = "nifi.c2.file.watch";
+const char *Configure::nifi_c2_flow_id = "nifi.c2.flow.id";
+const char *Configure::nifi_c2_flow_url = "nifi.c2.flow.url";
 
 } /* namespace minifi */
 } /* namespace nifi */
diff --git a/libminifi/src/FlowController.cpp b/libminifi/src/FlowController.cpp
index 8ee15d49..1b6b073b 100644
--- a/libminifi/src/FlowController.cpp
+++ b/libminifi/src/FlowController.cpp
@@ -53,6 +53,7 @@
 #include "core/logging/LoggerConfiguration.h"
 #include "core/Connectable.h"
 #include "utils/HTTPClient.h"
+#include "io/NetworkPrioritizer.h"
 
 #ifdef _MSC_VER
 #ifndef PATH_MAX
@@ -100,6 +101,7 @@ FlowController::FlowController(std::shared_ptr<core::Repository> provenance_repo
   id_generator_->generate(uuid_);
   setUUID(uuid_);
 
+  flow_update_ = false;
   // Setup the default values
   if (flow_configuration_ != nullptr) {
     configuration_filename_ = flow_configuration_->getConfigurationPath();
@@ -185,13 +187,26 @@ bool FlowController::applyConfiguration(const std::string &source, const std::st
   std::lock_guard<std::recursive_mutex> flow_lock(mutex_);
   stop(true);
   waitUnload(30000);
+  controller_map_->clear();
   this->root_ = std::move(newRoot);
-  loadFlowRepo();
-  initialized_ = true;
+  initialized_ = false;
+  load(this->root_, true);
+  flow_update_ = true;
   bool started = start() == 0;
 
   updating_ = false;
 
+  if (started) {
+    auto flowVersion = flow_configuration_->getFlowVersion();
+    if (flowVersion) {
+      logger_->log_debug("Setting flow id to %s", flowVersion->getFlowId());
+      configuration_->set(Configure::nifi_c2_flow_id, flowVersion->getFlowId());
+      configuration_->set(Configure::nifi_c2_flow_url, flowVersion->getFlowIdentifier()->getRegistryUrl());
+    } else {
+      logger_->log_debug("Invalid flow version, not setting");
+    }
+  }
+
   return started;
 }
 
@@ -249,15 +264,23 @@ void FlowController::unload() {
   return;
 }
 
-void FlowController::load() {
+void FlowController::load(const std::shared_ptr<core::ProcessGroup> &root, bool reload) {
   std::lock_guard<std::recursive_mutex> flow_lock(mutex_);
   if (running_) {
     stop(true);
   }
   if (!initialized_) {
-    logger_->log_info("Load Flow Controller from file %s", configuration_filename_.c_str());
+    if (root) {
+      logger_->log_info("Load Flow Controller from provided root");
+    } else {
+      logger_->log_info("Load Flow Controller from file %s", configuration_filename_.c_str());
+    }
 
-    this->root_ = std::shared_ptr<core::ProcessGroup>(flow_configuration_->getRoot(configuration_filename_));
+    if (reload) {
+      io::NetworkPrioritizerFactory::getInstance()->clearPrioritizer();
+    }
+
+    this->root_ = root == nullptr ? std::shared_ptr<core::ProcessGroup>(flow_configuration_->getRoot(configuration_filename_)) : root;
 
     logger_->log_info("Loaded root processor Group");
 
@@ -265,12 +288,12 @@ void FlowController::load() {
 
     controller_service_provider_ = flow_configuration_->getControllerServiceProvider();
 
-    if (nullptr == timer_scheduler_) {
+    if (nullptr == timer_scheduler_ || reload) {
       timer_scheduler_ = std::make_shared<TimerDrivenSchedulingAgent>(
           std::static_pointer_cast<core::controller::ControllerServiceProvider>(std::dynamic_pointer_cast<FlowController>(shared_from_this())), provenance_repo_, flow_file_repo_, content_repo_,
           configuration_);
     }
-    if (nullptr == event_scheduler_) {
+    if (nullptr == event_scheduler_ || reload) {
       event_scheduler_ = std::make_shared<EventDrivenSchedulingAgent>(
           std::static_pointer_cast<core::controller::ControllerServiceProvider>(std::dynamic_pointer_cast<FlowController>(shared_from_this())), provenance_repo_, flow_file_repo_, content_repo_,
           configuration_);
@@ -352,8 +375,6 @@ void FlowController::initializeC2() {
   if (!c2_enabled_) {
     return;
   }
-  if (c2_initialized_)
-    return;
 
   std::string c2_enable_str;
   std::string class_str;
@@ -393,16 +414,22 @@ void FlowController::initializeC2() {
     // set to the flow controller's identifier
     identifier_str = uuidStr_;
   }
-  configuration_->setAgentIdentifier(identifier_str);
-  state::StateManager::initialize();
 
-  std::shared_ptr<c2::C2Agent> agent = std::make_shared<c2::C2Agent>(std::dynamic_pointer_cast<FlowController>(shared_from_this()), std::dynamic_pointer_cast<FlowController>(shared_from_this()),
-                                                                     configuration_);
-  registerUpdateListener(agent, agent->getHeartBeatDelay());
+  if (!c2_initialized_) {
+    configuration_->setAgentIdentifier(identifier_str);
+    state::StateManager::initialize();
+    std::shared_ptr<c2::C2Agent> agent = std::make_shared<c2::C2Agent>(std::dynamic_pointer_cast<FlowController>(shared_from_this()), std::dynamic_pointer_cast<FlowController>(shared_from_this()),
+                                                                       configuration_);
+    registerUpdateListener(agent, agent->getHeartBeatDelay());
 
-  state::StateManager::startMetrics(agent->getHeartBeatDelay());
+    state::StateManager::startMetrics(agent->getHeartBeatDelay());
 
-  c2_initialized_ = true;
+    c2_initialized_ = true;
+  } else {
+    if (!flow_update_) {
+      return;
+    }
+  }
   device_information_.clear();
   component_metrics_.clear();
   component_metrics_by_id_.clear();
diff --git a/libminifi/src/Properties.cpp b/libminifi/src/Properties.cpp
index c8cb341b..81bb38c0 100644
--- a/libminifi/src/Properties.cpp
+++ b/libminifi/src/Properties.cpp
@@ -18,6 +18,7 @@
 #include "properties/Properties.h"
 #include <string>
 #include "utils/StringUtils.h"
+#include "utils/file/FileUtils.h"
 #include "core/Core.h"
 #include "core/logging/LoggerConfiguration.h"
 
@@ -76,7 +77,7 @@ int Properties::getInt(const std::string &key, int default_value) {
 }
 
 // Parse one line in configure file like key=value
-void Properties::parseConfigureFileLine(char *buf) {
+bool Properties::parseConfigureFileLine(char *buf, std::string &prop_key, std::string &prop_value) {
   char *line = buf;
 
   while ((line[0] == ' ') || (line[0] == '\t'))
@@ -84,12 +85,12 @@ void Properties::parseConfigureFileLine(char *buf) {
 
   char first = line[0];
   if ((first == '\0') || (first == '#') || (first == '\r') || (first == '\n') || (first == '=')) {
-    return;
+    return true;
   }
 
   char *equal = strchr(line, '=');
   if (equal == NULL) {
-    return;
+    return false;  // invalid property as this is not a comment or property line
   }
 
   equal[0] = '\0';
@@ -101,14 +102,14 @@ void Properties::parseConfigureFileLine(char *buf) {
 
   first = equal[0];
   if ((first == '\0') || (first == '\r') || (first == '\n')) {
-    return;
+    return true;  // empty properties are okay
   }
 
   std::string value = equal;
   value = org::apache::nifi::minifi::utils::StringUtils::replaceEnvironmentVariables(value);
-  key = org::apache::nifi::minifi::utils::StringUtils::trimRight(key);
-  value = org::apache::nifi::minifi::utils::StringUtils::trimRight(value);
-  set(key, value);
+  prop_key = org::apache::nifi::minifi::utils::StringUtils::trimRight(key);
+  prop_value = org::apache::nifi::minifi::utils::StringUtils::trimRight(value);
+  return true;
 }
 
 // Load Configure File
@@ -129,7 +130,9 @@ void Properties::loadConfigureFile(const char *fileName) {
 #else
   path = const_cast<char*>(adjustedFilename.c_str());
 #endif
-  logger_->log_info("Using configuration file located at %s", path);
+  logger_->log_info("Using configuration file located at %s, from %s", path, fileName);
+
+  properties_file_ = path;
 
   std::ifstream file(path, std::ifstream::in);
   if (!file.good()) {
@@ -140,8 +143,110 @@ void Properties::loadConfigureFile(const char *fileName) {
 
   char buf[TRACE_BUFFER_SIZE];
   for (file.getline(buf, TRACE_BUFFER_SIZE); file.good(); file.getline(buf, TRACE_BUFFER_SIZE)) {
-    parseConfigureFileLine(buf);
+    std::string key, value;
+    if (parseConfigureFileLine(buf, key, value)) {
+      set(key, value);
+    }
+  }
+  dirty_ = false;
+}
+
+bool Properties::validateConfigurationFile(const std::string &configFile) {
+  std::ifstream file(configFile, std::ifstream::in);
+  if (!file.good()) {
+    logger_->log_error("load configure file failed %s", configFile);
+    return false;
+  }
+
+  char buf[TRACE_BUFFER_SIZE];
+  for (file.getline(buf, TRACE_BUFFER_SIZE); file.good(); file.getline(buf, TRACE_BUFFER_SIZE)) {
+    std::string key, value;
+    if (!parseConfigureFileLine(buf, key, value)) {
+      return false;
+    }
+  }
+  return true;
+}
+
+bool Properties::persistProperties() {
+  std::lock_guard<std::mutex> lock(mutex_);
+  if (!dirty_) {
+    logger_->log_info("Attempt to persist, but properties are not updated");
+    return true;
+  }
+  std::ifstream file(properties_file_, std::ifstream::in);
+  if (!file.good()) {
+    logger_->log_error("load configure file failed %s", properties_file_);
+    return false;
+  }
+
+  std::map<std::string, std::string> properties_copy = properties_;
+
+  std::string new_file = properties_file_ + ".new";
+
+  std::ofstream output_file(new_file, std::ios::out);
+
+  char buf[TRACE_BUFFER_SIZE];
+  for (file.getline(buf, TRACE_BUFFER_SIZE); file.good(); file.getline(buf, TRACE_BUFFER_SIZE)) {
+    char *line = buf;
+
+    char first = line[0];
+
+    if ((first == '\0') || (first == '#') || (first == '\r') || (first == '\n') || (first == '=')) {
+      // persist comments and newlines
+      output_file << line << std::endl;
+      continue;
+    }
+
+    char *equal = strchr(line, '=');
+    if (equal == NULL) {
+      output_file << line << std::endl;
+      continue;
+    }
+
+    equal[0] = '\0';
+    std::string key = line;
+
+    equal++;
+    while ((equal[0] == ' ') || (equal[0] == '\t'))
+      ++equal;
+
+    first = equal[0];
+    if ((first == '\0') || (first == '\r') || (first == '\n')) {
+      output_file << line << std::endl;
+      continue;
+    }
+
+    std::string value = equal;
+    value = org::apache::nifi::minifi::utils::StringUtils::replaceEnvironmentVariables(value);
+    key = org::apache::nifi::minifi::utils::StringUtils::trimRight(key);
+    value = org::apache::nifi::minifi::utils::StringUtils::trimRight(value);
+    auto hasIt = properties_copy.find(key);
+    if (hasIt != properties_copy.end() && !value.empty()) {
+      output_file << key << "=" << hasIt->second << std::endl;
+    }
+    properties_copy.erase(key);
+  }
+
+  for (const auto &kv : properties_copy) {
+    if (!kv.first.empty() && !kv.second.empty())
+      output_file << kv.first << "=" << kv.second << std::endl;
   }
+  output_file.close();
+
+  if (validateConfigurationFile(new_file)) {
+    const std::string backup = properties_file_ + ".bak";
+    if (!utils::file::FileUtils::copy_file(properties_file_, backup) && !utils::file::FileUtils::copy_file(new_file, properties_file_)) {
+      logger_->log_info("Persisted %s", properties_file_);
+      return true;
+    } else {
+      logger_->log_error("Could not update %s", properties_file_);
+    }
+  }
+
+  dirty_ = false;
+
+  return false;
 }
 
 // Parse Command Line
diff --git a/libminifi/src/SchedulingAgent.cpp b/libminifi/src/SchedulingAgent.cpp
index ca303162..d9c1ab5a 100644
--- a/libminifi/src/SchedulingAgent.cpp
+++ b/libminifi/src/SchedulingAgent.cpp
@@ -42,10 +42,10 @@ bool SchedulingAgent::hasWorkToDo(std::shared_ptr<core::Processor> processor) {
 std::future<uint64_t> SchedulingAgent::enableControllerService(std::shared_ptr<core::controller::ControllerServiceNode> &serviceNode) {
   logger_->log_info("Enabling CSN in SchedulingAgent %s", serviceNode->getName());
   // reference the enable function from serviceNode
-  std::function< uint64_t()> f_ex = [serviceNode] {
-    serviceNode->enable();
-    return 0;
-  };
+  std::function<uint64_t()> f_ex = [serviceNode] {
+      serviceNode->enable();
+      return 0;
+    };
 
   // only need to run this once.
   std::unique_ptr<SingleRunMonitor> monitor = std::unique_ptr<SingleRunMonitor>(new SingleRunMonitor(&running_));
@@ -54,14 +54,15 @@ std::future<uint64_t> SchedulingAgent::enableControllerService(std::shared_ptr<c
   // we aren't terribly concerned with the result.
   std::future<uint64_t> future;
   thread_pool_.execute(std::move(functor), future);
-  future.wait();
+  if (future.valid())
+    future.wait();
   return future;
 }
 
 std::future<uint64_t> SchedulingAgent::disableControllerService(std::shared_ptr<core::controller::ControllerServiceNode> &serviceNode) {
   logger_->log_info("Disabling CSN in SchedulingAgent %s", serviceNode->getName());
   // reference the disable function from serviceNode
-  std::function< uint64_t()> f_ex = [serviceNode] {
+  std::function<uint64_t()> f_ex = [serviceNode] {
     serviceNode->disable();
     return 0;
   };
@@ -74,7 +75,8 @@ std::future<uint64_t> SchedulingAgent::disableControllerService(std::shared_ptr<
   // we aren't terribly concerned with the result.
   std::future<uint64_t> future;
   thread_pool_.execute(std::move(functor), future);
-  future.wait();
+  if (future.valid())
+    future.wait();
   return future;
 }
 
diff --git a/libminifi/src/c2/C2Agent.cpp b/libminifi/src/c2/C2Agent.cpp
index 8168c77f..ea5802f5 100644
--- a/libminifi/src/c2/C2Agent.cpp
+++ b/libminifi/src/c2/C2Agent.cpp
@@ -19,6 +19,7 @@
 #include "c2/C2Agent.h"
 #include <csignal>
 #include <utility>
+#include <limits>
 #include <vector>
 #include <map>
 #include <string>
@@ -55,6 +56,10 @@ C2Agent::C2Agent(const std::shared_ptr<core::controller::ControllerServiceProvid
     update_service_ = std::static_pointer_cast<controllers::UpdatePolicyControllerService>(controller_->getControllerService(C2_AGENT_UPDATE_NAME));
   }
 
+  if (update_service_ == nullptr) {
+    // create a stubbed service for updating the flow identifier
+  }
+
   configure(configuration, false);
 
   c2_producer_ = [&]() {
@@ -82,7 +87,7 @@ C2Agent::C2Agent(const std::shared_ptr<core::controller::ControllerServiceProvid
 
       checkTriggers();
 
-      std::this_thread::sleep_for(std::chrono::milliseconds(500));
+      std::this_thread::sleep_for(std::chrono::milliseconds(heart_beat_period_ > 500 ? 500 : heart_beat_period_));
       return state::Update(state::UpdateStatus(state::UpdateState::READ_COMPLETE, false));
     };
 
@@ -306,21 +311,23 @@ void C2Agent::performHeartBeat() {
   }
 }
 
-void C2Agent::serializeMetrics(C2Payload &metric_payload, const std::string &name, const std::vector<state::response::SerializedResponseNode> &metrics, bool is_container) {
-  for (auto metric : metrics) {
+void C2Agent::serializeMetrics(C2Payload &metric_payload, const std::string &name, const std::vector<state::response::SerializedResponseNode> &metrics, bool is_container, bool is_collapsible) {
+  for (const auto &metric : metrics) {
     if (metric.children.size() > 0) {
       C2Payload child_metric_payload(metric_payload.getOperation());
       if (metric.array) {
         child_metric_payload.setContainer(true);
       }
+      auto collapsible = !metric.collapsible ? metric.collapsible : is_collapsible;
+      child_metric_payload.setCollapsible(collapsible);
       child_metric_payload.setLabel(metric.name);
-      serializeMetrics(child_metric_payload, metric.name, metric.children, is_container);
+      serializeMetrics(child_metric_payload, metric.name, metric.children, is_container, collapsible);
       metric_payload.addPayload(std::move(child_metric_payload));
     } else {
       C2ContentResponse response(metric_payload.getOperation());
       response.name = name;
       response.operation_arguments[metric.name] = metric.value;
-      metric_payload.addContent(std::move(response));
+      metric_payload.addContent(std::move(response), is_collapsible);
     }
   }
 }
@@ -633,15 +640,20 @@ void C2Agent::handle_update(const C2ContentResponse &resp) {
               writer.write(raw_data_str.data(), raw_data_str.size());
             }
             writer.close();
+
+            // update the flow id
+            configuration_->persistProperties();
           }
         }
       } else {
         logger_->log_debug("update failed.");
+        std::cout << raw_data_str << std::endl;
         C2Payload response(Operation::ACKNOWLEDGE, resp.ident, false, true);
         enqueue_c2_response(std::move(response));
       }
       // send
     } else {
+      logger_->log_debug("Did not have location within %s", resp.ident);
       auto update_text = resp.operation_arguments.find("configuration_data");
       if (update_text != resp.operation_arguments.end()) {
         if (update_sink_->applyUpdate(url->second.to_string(), update_text->second.to_string()) != 0 && persist != resp.operation_arguments.end()
@@ -679,14 +691,10 @@ void C2Agent::handle_update(const C2ContentResponse &resp) {
   } else if (resp.name == "properties") {
     bool update_occurred = false;
     for (auto entry : resp.operation_arguments) {
-      if (nullptr != update_service_) {
-        if (update_service_->canUpdate(entry.first)) {
-          configuration_->set(entry.first, entry.second.to_string());
-          update_occurred = true;
-        }
-      }
+      if (update_property(entry.first, entry.second.to_string()))
+        update_occurred = true;
     }
-    if (nullptr != update_service_ && update_occurred) {
+    if (update_occurred) {
       // enable updates to persist the configuration.
     }
   } else if (resp.name == "c2") {
@@ -747,6 +755,20 @@ void C2Agent::handle_update(const C2ContentResponse &resp) {
   }
 }
 
+/**
+ * Updates a property
+ */
+bool C2Agent::update_property(const std::string &property_name, const std::string &property_value, bool persist) {
+  if (update_service_->canUpdate(property_name)) {
+    configuration_->set(property_name, property_value);
+    if (persist) {
+      configuration_->persistProperties();
+      return true;
+    }
+  }
+  return false;
+}
+
 void C2Agent::restart_agent() {
   char cwd[1024];
   if (getcwd(cwd, sizeof(cwd)) == nullptr) {
diff --git a/libminifi/src/c2/C2Payload.cpp b/libminifi/src/c2/C2Payload.cpp
index 91518b33..405de737 100644
--- a/libminifi/src/c2/C2Payload.cpp
+++ b/libminifi/src/c2/C2Payload.cpp
@@ -81,7 +81,8 @@ C2Payload::C2Payload(Operation op, std::string identifier, bool resp, bool isRaw
       raw_(isRaw),
       ident_(identifier),
       isResponse(resp),
-      is_container_(false) {
+      is_container_(false),
+      is_collapsible_(true) {
 }
 
 C2Payload::C2Payload(Operation op, bool resp, bool isRaw)
@@ -89,7 +90,8 @@ C2Payload::C2Payload(Operation op, bool resp, bool isRaw)
       op_(op),
       raw_(isRaw),
       isResponse(resp),
-      is_container_(false) {
+      is_container_(false),
+      is_collapsible_(true) {
 }
 
 C2Payload::C2Payload(Operation op, state::UpdateState state, bool resp, bool isRaw)
@@ -97,33 +99,8 @@ C2Payload::C2Payload(Operation op, state::UpdateState state, bool resp, bool isR
       op_(op),
       raw_(isRaw),
       isResponse(resp),
-      is_container_(false) {
-}
-
-C2Payload::C2Payload(const C2Payload &other)
-    : state::Update(other),
-      isResponse(other.isResponse),
-      op_(other.op_),
-      raw_(other.raw_),
-      label_(other.label_),
-      ident_(other.ident_),
-      raw_data_(other.raw_data_),
-      payloads_(other.payloads_),
-      content_(other.content_),
-      is_container_(other.is_container_) {
-}
-
-C2Payload::C2Payload(const C2Payload &&other)
-    : state::Update(std::move(other)),
-      isResponse(other.isResponse),
-      op_(std::move(other.op_)),
-      raw_(other.raw_),
-      label_(std::move(other.label_)),
-      ident_(std::move(other.ident_)),
-      raw_data_(std::move(other.raw_data_)),
-      payloads_(std::move(other.payloads_)),
-      content_(std::move(other.content_)),
-      is_container_(std::move(other.is_container_)) {
+      is_container_(false),
+      is_collapsible_(true) {
 }
 
 void C2Payload::setIdentifier(const std::string &ident) {
@@ -146,21 +123,13 @@ const std::vector<C2ContentResponse> &C2Payload::getContent() const {
   return content_;
 }
 
-void C2Payload::addContent(const C2ContentResponse &&content) {
-  for (auto &existing_content : content_) {
-    if (existing_content.name == content.name) {
-      for (auto subcontent : existing_content.operation_arguments) {
+void C2Payload::addContent(const C2ContentResponse &&content, bool collapsible) {
+  if (collapsible) {
+    for (auto &existing_content : content_) {
+      if (existing_content.name == content.name) {
+        existing_content.operation_arguments.insert(content.operation_arguments.begin(), content.operation_arguments.end());
+        return;
       }
-
-      for (auto subcontent : content.operation_arguments) {
-      }
-
-      existing_content.operation_arguments.insert(content.operation_arguments.begin(), content.operation_arguments.end());
-
-      for (auto subcontent : existing_content.operation_arguments) {
-      }
-
-      return;
     }
   }
   content_.push_back(std::move(content));
@@ -179,12 +148,11 @@ void C2Payload::setRawData(const std::vector<char> &data) {
 }
 
 void C2Payload::setRawData(const std::vector<uint8_t> &data) {
-  std::transform(std::begin(data), std::end(data), std::back_inserter(raw_data_), [](uint8_t c){
+  std::transform(std::begin(data), std::end(data), std::back_inserter(raw_data_), [](uint8_t c) {
     return static_cast<char>(c);
   });
 }
 
-
 std::vector<char> C2Payload::getRawData() const {
   return raw_data_;
 }
@@ -196,36 +164,6 @@ const std::vector<C2Payload> &C2Payload::getNestedPayloads() const {
   return payloads_;
 }
 
-C2Payload &C2Payload::operator=(const C2Payload &&other) {
-  state::Update::operator=(std::move(other));
-  isResponse = other.isResponse;
-  op_ = std::move(other.op_);
-  raw_ = other.raw_;
-  if (raw_) {
-    raw_data_ = std::move(other.raw_data_);
-  }
-  label_ = std::move(other.label_);
-  payloads_ = std::move(other.payloads_);
-  content_ = std::move(other.content_);
-  is_container_ = std::move(other.is_container_);
-  return *this;
-}
-
-C2Payload &C2Payload::operator=(const C2Payload &other) {
-  state::Update::operator=(other);
-  isResponse = other.isResponse;
-  op_ = other.op_;
-  raw_ = other.raw_;
-  if (raw_) {
-    raw_data_ = other.raw_data_;
-  }
-  label_ = other.label_;
-  payloads_ = other.payloads_;
-  content_ = other.content_;
-  is_container_ = other.is_container_;
-  return *this;
-}
-
 } /* namespace c2 */
 } /* namespace minifi */
 } /* namespace nifi */
diff --git a/libminifi/src/c2/protocols/RESTProtocol.cpp b/libminifi/src/c2/protocols/RESTProtocol.cpp
index 76dbae1a..297b76d0 100644
--- a/libminifi/src/c2/protocols/RESTProtocol.cpp
+++ b/libminifi/src/c2/protocols/RESTProtocol.cpp
@@ -18,6 +18,7 @@
 
 #include "c2/protocols/RESTProtocol.h"
 
+#include "core/TypedValues.h"
 #include <algorithm>
 #include <memory>
 #include <utility>
@@ -138,16 +139,33 @@ void setJsonStr(const std::string& key, const state::response::ValueNode& value,
   auto base_type = value.getValue();
   keyVal.SetString(c_key, key.length(), alloc);
 
-  if (auto sub_type = std::dynamic_pointer_cast<state::response::IntValue>(base_type)) {
-    valueVal.SetInt(sub_type->getValue());
-  } else if (auto sub_type = std::dynamic_pointer_cast<state::response::Int64Value>(base_type)) {
-    valueVal.SetInt64(sub_type->getValue());
-  } else if (auto sub_type = std::dynamic_pointer_cast<state::response::BoolValue>(base_type)) {
-    valueVal.SetBool(sub_type->getValue());
-  } else {
+  auto type_index = base_type->getTypeIndex();
+  if (auto sub_type = std::dynamic_pointer_cast<core::TransformableValue>(base_type)) {
     auto str = base_type->getStringValue();
     const char* c_val = str.c_str();
     valueVal.SetString(c_val, str.length(), alloc);
+  } else {
+    if (type_index == state::response::Value::BOOL_TYPE) {
+      bool value = false;
+      base_type->convertValue(value);
+      valueVal.SetBool(value);
+    } else if (type_index == state::response::Value::INT_TYPE) {
+      int value = 0;
+      base_type->convertValue(value);
+      valueVal.SetInt(value);
+    } else if (type_index == state::response::Value::INT64_TYPE) {
+      int64_t value = 0;
+      base_type->convertValue(value);
+      valueVal.SetInt64(value);
+    } else if (type_index == state::response::Value::UINT64_TYPE) {
+      int64_t value = 0;
+      base_type->convertValue(value);
+      valueVal.SetInt64(value);
+    } else {
+      auto str = base_type->getStringValue();
+      const char* c_val = str.c_str();
+      valueVal.SetString(c_val, str.length(), alloc);
+    }
   }
   parent.AddMember(keyVal, valueVal, alloc);
 }
@@ -184,10 +202,11 @@ void RESTProtocol::mergePayloadContent(rapidjson::Value &target, const C2Payload
       for (const auto& op_arg : payload_content.operation_arguments) {
         rapidjson::Value keyVal;
         keyVal.SetString(op_arg.first.c_str(), op_arg.first.length(), alloc);
-        if (is_parent_array)
+        if (is_parent_array) {
           target.PushBack(keyVal, alloc);
-        else
+        } else {
           arr.PushBack(keyVal, alloc);
+        }
       }
     }
 
@@ -261,20 +280,40 @@ rapidjson::Value RESTProtocol::serializeJsonPayload(const C2Payload &payload, ra
 // get the name from the content
   rapidjson::Value json_payload(payload.isContainer() ? rapidjson::kArrayType : rapidjson::kObjectType);
 
-  std::map<std::string, std::list<rapidjson::Value*>> children;
+  std::vector<ValueObject> children;
 
   for (const auto &nested_payload : payload.getNestedPayloads()) {
     rapidjson::Value* child_payload = new rapidjson::Value(serializeJsonPayload(nested_payload, alloc));
 
-    children[nested_payload.getLabel()].push_back(child_payload);
+    if (nested_payload.isCollapsible()) {
+      bool combine = false;
+      for (auto &subordinate : children) {
+        if (subordinate.name == nested_payload.getLabel()) {
+          subordinate.values.push_back(child_payload);
+          combine = true;
+          break;
+        }
+      }
+      if (!combine) {
+        ValueObject obj;
+        obj.name = nested_payload.getLabel();
+        obj.values.push_back(child_payload);
+        children.push_back(obj);
+      }
+    } else {
+      ValueObject obj;
+      obj.name = nested_payload.getLabel();
+      obj.values.push_back(child_payload);
+      children.push_back(obj);
+    }
   }
 
   for (auto child_vector : children) {
     rapidjson::Value children_json;
-    rapidjson::Value newMemberKey = getStringValue(child_vector.first, alloc);
-    if (child_vector.second.size() > 1) {
+    rapidjson::Value newMemberKey = getStringValue(child_vector.name, alloc);
+    if (child_vector.values.size() > 1) {
       children_json.SetArray();
-      for (auto child : child_vector.second) {
+      for (auto child : child_vector.values) {
         if (json_payload.IsArray())
           json_payload.PushBack(child->Move(), alloc);
         else
@@ -282,8 +321,8 @@ rapidjson::Value RESTProtocol::serializeJsonPayload(const C2Payload &payload, ra
       }
       if (!json_payload.IsArray())
         json_payload.AddMember(newMemberKey, children_json, alloc);
-    } else if (child_vector.second.size() == 1) {
-      rapidjson::Value* first = child_vector.second.front();
+    } else if (child_vector.values.size() == 1) {
+      rapidjson::Value* first = child_vector.values.front();
       if (first->IsObject() && first->HasMember(newMemberKey)) {
         if (json_payload.IsArray())
           json_payload.PushBack((*first)[newMemberKey].Move(), alloc);
@@ -297,7 +336,7 @@ rapidjson::Value RESTProtocol::serializeJsonPayload(const C2Payload &payload, ra
         }
       }
     }
-    for (rapidjson::Value* child : child_vector.second)
+    for (rapidjson::Value* child : child_vector.values)
       delete child;
   }
 
diff --git a/libminifi/src/controllers/LinuxPowerManagementService.cpp b/libminifi/src/controllers/LinuxPowerManagementService.cpp
index ae0651b0..26f82c83 100644
--- a/libminifi/src/controllers/LinuxPowerManagementService.cpp
+++ b/libminifi/src/controllers/LinuxPowerManagementService.cpp
@@ -28,12 +28,29 @@ namespace nifi {
 namespace minifi {
 namespace controllers {
 
-core::Property LinuxPowerManagerService::BatteryCapacityPath("Battery Capacity Path", "Path to the battery level");
-core::Property LinuxPowerManagerService::BatteryStatusPath("Battery Status Path", "Path to the battery status ( Discharging/Battery )");
-core::Property LinuxPowerManagerService::BatteryStatusDischargeKeyword("Battery Status Discharge", "Keyword to identify if battery is discharging");
-core::Property LinuxPowerManagerService::TriggerThreshold("Trigger Threshold", "Battery threshold before which we consider a slow reduction");
-core::Property LinuxPowerManagerService::LowBatteryThreshold("Low Battery Threshold", "Battery threshold before which we will aggressively reduce");
-core::Property LinuxPowerManagerService::WaitPeriod("Wait Period", "Decay between checking threshold and determining if a reduction is needed");
+core::Property LinuxPowerManagerService::BatteryCapacityPath(
+    core::PropertyBuilder::createProperty("Battery Capacity Path")->withDescription("Path to the battery level")->isRequired(true)->withDefaultValue<std::string>(
+        "/sys/class/power_supply/BAT0/capacity")->build());
+
+core::Property LinuxPowerManagerService::BatteryStatusPath(
+    core::PropertyBuilder::createProperty("Battery Status Path")->withDescription("Path to the battery status ( Discharging/Battery )")->isRequired(true)->withDefaultValue<std::string>(
+        "/sys/class/power_supply/BAT0/status")->build());
+
+core::Property LinuxPowerManagerService::BatteryStatusDischargeKeyword(
+    core::PropertyBuilder::createProperty("Battery Status Discharge")->withDescription("Keyword to identify if battery is discharging")->isRequired(true)->withDefaultValue<std::string>("Discharging")
+        ->build());
+
+core::Property LinuxPowerManagerService::TriggerThreshold(
+    core::PropertyBuilder::createProperty("Trigger Threshold")->withDescription("Battery threshold before which we consider a slow reduction. Should be a number from 1-100")->isRequired(true)
+        ->withDefaultValue<int>(75)->build());
+
+core::Property LinuxPowerManagerService::WaitPeriod(
+    core::PropertyBuilder::createProperty("Wait Period")->withDescription("Decay between checking threshold and determining if a reduction is needed")->isRequired(true)
+        ->withDefaultValue<core::TimePeriodValue>("100 msec")->build());
+
+core::Property LinuxPowerManagerService::LowBatteryThreshold(
+    core::PropertyBuilder::createProperty("Low Battery Threshold")->withDescription("Battery threshold before which we will aggressively reduce. Should be a number from 1-100")->isRequired(true)
+        ->withDefaultValue<int>(50)->build());
 
 bool LinuxPowerManagerService::isAboveMax(int new_tasks) {
   return false;
@@ -45,16 +62,20 @@ uint16_t LinuxPowerManagerService::getMaxThreads() {
 
 bool LinuxPowerManagerService::canIncrease() {
   for (const auto path_pair : paths_) {
-    auto capacity = path_pair.first;
-    auto status = path_pair.second;
+    try {
+      auto capacity = path_pair.first;
+      auto status = path_pair.second;
 
-    std::ifstream status_file(status);
-    std::string status_str;
-    std::getline(status_file, status_str);
-    status_file.close();
+      std::ifstream status_file(status);
+      std::string status_str;
+      std::getline(status_file, status_str);
+      status_file.close();
 
-    if (!utils::StringUtils::equalsIgnoreCase(status_keyword_, status_str)) {
-      return true;
+      if (!utils::StringUtils::equalsIgnoreCase(status_keyword_, status_str)) {
+        return true;
+      }
+    } catch (...) {
+      logger_->log_error("Could not read file paths. ignoring, temporarily");
     }
   }
   return false;
@@ -84,23 +105,28 @@ bool LinuxPowerManagerService::shouldReduce() {
 
   int battery_sum = 0;
   for (const auto path_pair : paths_) {
-    auto capacity = path_pair.first;
-    auto status = path_pair.second;
-
-    std::ifstream capacity_file(capacity);
-    std::string capacity_str;
-    std::getline(capacity_file, capacity_str);
-    capacity_file.close();
-    int battery_level = std::stoi(capacity_str);
-    battery_sum += battery_level;
-
-    std::ifstream status_file(status);
-    std::string status_str;
-    std::getline(status_file, status_str);
-    status_file.close();
-
-    if (!utils::StringUtils::equalsIgnoreCase(status_keyword_, status_str)) {
-      all_discharging &= false;
+    try {
+      auto capacity = path_pair.first;
+      auto status = path_pair.second;
+
+      std::ifstream capacity_file(capacity);
+      std::string capacity_str;
+      std::getline(capacity_file, capacity_str);
+      capacity_file.close();
+      int battery_level = std::stoi(capacity_str);
+      battery_sum += battery_level;
+
+      std::ifstream status_file(status);
+      std::string status_str;
+      std::getline(status_file, status_str);
+      status_file.close();
+
+      if (!utils::StringUtils::equalsIgnoreCase(status_keyword_, status_str)) {
+        all_discharging &= false;
+      }
+    } catch (...) {
+      logger_->log_error("Error caught while pulling paths");
+      return false;
     }
   }
 
@@ -159,25 +185,17 @@ void LinuxPowerManagerService::onEnable() {
     logger_->log_trace("Cannot enable Linux Power Manager");
     return;
   }
-  std::string trigger, wait;
   status_keyword_ = "Discharging";
   core::Property capacityPaths;
   core::Property statusPaths;
 
-  if (getProperty(TriggerThreshold.getName(), trigger) && getProperty(WaitPeriod.getName(), wait)) {
-    core::TimeUnit unit;
-    int64_t wait_time;
-    if (core::Property::StringToTime(wait, wait_time, unit) && core::Property::ConvertTimeUnitToMS(wait_time, unit, wait_time)) {
-      wait_period_ = wait_time;
-    }
+  uint64_t wait;
+  if (getProperty(TriggerThreshold.getName(), trigger_) && getProperty(WaitPeriod.getName(), wait)) {
+    wait_period_ = wait;
 
     getProperty(BatteryStatusDischargeKeyword.getName(), status_keyword_);
 
-    trigger_ = std::stoi(trigger);
-
-    if (getProperty(LowBatteryThreshold.getName(), trigger)) {
-      low_battery_trigger_ = std::stoi(trigger);
-    } else {
+    if (!getProperty(LowBatteryThreshold.getName(), low_battery_trigger_)) {
       low_battery_trigger_ = 0;
     }
     getProperty(BatteryCapacityPath.getName(), capacityPaths);
@@ -189,6 +207,7 @@ void LinuxPowerManagerService::onEnable() {
     } else {
       logger_->log_error("BatteryCapacityPath and BatteryStatusPath mis-configuration");
     }
+
     enabled_ = true;
     logger_->log_trace("Enabled enable ");
   } else {
diff --git a/libminifi/src/controllers/NetworkPrioritizerService.cpp b/libminifi/src/controllers/NetworkPrioritizerService.cpp
index d92758a6..5df8af2e 100644
--- a/libminifi/src/controllers/NetworkPrioritizerService.cpp
+++ b/libminifi/src/controllers/NetworkPrioritizerService.cpp
@@ -35,6 +35,7 @@
 
 #include <set>
 #include "utils/StringUtils.h"
+#include "core/TypedValues.h"
 #if ( defined(__APPLE__) || defined(__MACH__) || defined(BSD))
 #include <net/if_dl.h>
 #include <net/if_types.h>
@@ -46,11 +47,22 @@ namespace nifi {
 namespace minifi {
 namespace controllers {
 
-core::Property NetworkPrioritizerService::NetworkControllers("Network Controllers", "Network controllers in order of priority for this prioritizer");
-core::Property NetworkPrioritizerService::MaxThroughput("Max Throughput", "Max throughput for these network controllers");
-core::Property NetworkPrioritizerService::MaxPayload("Max Payload", "Maximum payload for these network controllers");
-core::Property NetworkPrioritizerService::VerifyInterfaces("Verify Interfaces", "Verify that interfaces are operational", "true");
-core::Property NetworkPrioritizerService::DefaultPrioritizer("Default Prioritizer", "Sets this controller service as the default prioritizer for all comms");
+core::Property NetworkPrioritizerService::NetworkControllers(
+    core::PropertyBuilder::createProperty("Network Controllers")->withDescription("Comma separated list of network controllers in order of priority for this prioritizer")->isRequired(false)->build());
+
+core::Property NetworkPrioritizerService::MaxThroughput(
+    core::PropertyBuilder::createProperty("Max Throughput")->withDescription("Max throughput ( per second ) for these network controllers")->isRequired(true)->withDefaultValue<core::DataSizeValue>(
+        "1 MB")->build());
+
+core::Property NetworkPrioritizerService::MaxPayload(
+    core::PropertyBuilder::createProperty("Max Payload")->withDescription("Maximum payload for these network controllers")->isRequired(true)->withDefaultValue<core::DataSizeValue>("1 GB")->build());
+
+core::Property NetworkPrioritizerService::VerifyInterfaces(
+    core::PropertyBuilder::createProperty("Verify Interfaces")->withDescription("Verify that interfaces are operational")->isRequired(true)->withDefaultValue<bool>(true)->build());
+
+core::Property NetworkPrioritizerService::DefaultPrioritizer(
+    core::PropertyBuilder::createProperty("Default Prioritizer")->withDescription("Sets this controller service as the default prioritizer for all comms")->isRequired(false)->withDefaultValue<bool>(
+        false)->build());
 
 void NetworkPrioritizerService::initialize() {
   std::set<core::Property> supportedProperties;
@@ -117,10 +129,12 @@ std::string NetworkPrioritizerService::get_nearest_interface(const std::vector<s
 bool NetworkPrioritizerService::interface_online(const std::string &ifc) {
 #ifndef WIN32
   struct ifreq ifr;
-  auto sockid = socket(PF_INET6, SOCK_DGRAM, IPPROTO_IP);
+  auto sockid = socket(PF_INET, SOCK_DGRAM, IPPROTO_IP);
   memset(&ifr, 0, sizeof(ifr));
-  snprintf(ifr.ifr_name, ifc.length(), "%s", ifc.c_str());
+  memcpy(ifr.ifr_name, ifc.data(), ifc.length());
+  ifr.ifr_name[ifc.length()] = 0;
   if (ioctl(sockid, SIOCGIFFLAGS, &ifr) < 0) {
+    logger_->log_trace("Could not use ioctl on %s", ifc);
     return false;
   }
   close(sockid);
@@ -175,12 +189,12 @@ bool NetworkPrioritizerService::isWorkAvailable() {
 }
 
 void NetworkPrioritizerService::onEnable() {
-  std::string controllers, max_throughput, max_payload, df_prioritizer, intersect, verify_interfaces, roundrobin_interfaces;
+  std::string controllers, max_throughput, max_payload_, df_prioritizer, intersect, verify_interfaces, roundrobin_interfaces;
 // if we have defined controller services or we have linked services
   if (getProperty(NetworkControllers.getName(), controllers) || !linked_services_.empty()) {
     // if this controller service is defined, it will be an intersection of this config with linked services.
-    if (getProperty(MaxThroughput.getName(), max_throughput) && !max_throughput.empty()) {
-      max_throughput_ = std::stoi(max_throughput);
+    if (getProperty(MaxThroughput.getName(), max_throughput_)) {
+      logger_->log_trace("Max throughput is %d", max_throughput_);
       if (max_throughput_ < 1000) {
         bytes_per_token_ = 1;
         tokens_ = max_throughput_;
@@ -189,29 +203,26 @@ void NetworkPrioritizerService::onEnable() {
       }
     }
 
-    if (getProperty(MaxPayload.getName(), max_payload) && !max_payload.empty()) {
-      max_payload_ = std::stoi(max_payload);
-    }
+    getProperty(MaxPayload.getName(), max_payload_);
 
     if (!controllers.empty()) {
       network_controllers_ = utils::StringUtils::split(controllers, ",");
+      for (const auto &ifc : network_controllers_) {
+        logger_->log_trace("%s added to list of applied interfaces", ifc);
+      }
     }
-    if (getProperty(DefaultPrioritizer.getName(), df_prioritizer)) {
-      bool is_default = false;
-      if (utils::StringUtils::StringToBool(df_prioritizer, is_default)) {
-        if (is_default) {
-          if (io::NetworkPrioritizerFactory::getInstance()->setPrioritizer(shared_from_this()) < 0) {
-            std::runtime_error("Can only have one prioritizer");
-          }
+    bool is_default = false;
+    if (getProperty(DefaultPrioritizer.getName(), is_default)) {
+      if (is_default) {
+        if (io::NetworkPrioritizerFactory::getInstance()->setPrioritizer(shared_from_this()) < 0) {
+          std::runtime_error("Can only have one prioritizer");
         }
       }
     }
-    if (getProperty(VerifyInterfaces.getName(), verify_interfaces)) {
-      utils::StringUtils::StringToBool(verify_interfaces, verify_interfaces_);
-    }
+    getProperty(VerifyInterfaces.getName(), verify_interfaces_);
     timestamp_ = std::chrono::system_clock::now().time_since_epoch() / std::chrono::milliseconds(1);
     enabled_ = true;
-    logger_->log_trace("Enabled enable ");
+    logger_->log_trace("Enabled");
   } else {
     logger_->log_trace("Could not enable ");
   }
diff --git a/libminifi/src/controllers/UpdatePolicyControllerService.cpp b/libminifi/src/controllers/UpdatePolicyControllerService.cpp
index ae56fe11..fa485b71 100644
--- a/libminifi/src/controllers/UpdatePolicyControllerService.cpp
+++ b/libminifi/src/controllers/UpdatePolicyControllerService.cpp
@@ -30,6 +30,7 @@
 #include <net/if_types.h>
 #endif
 #include "core/state/UpdatePolicy.h"
+#include "core/PropertyValidation.h"
 
 namespace org {
 namespace apache {
@@ -37,10 +38,19 @@ namespace nifi {
 namespace minifi {
 namespace controllers {
 
-core::Property UpdatePolicyControllerService::AllowAllProperties("Allow All Properties", "Allows all properties, which are also not disallowed, to be updated", "false");
-core::Property UpdatePolicyControllerService::AllowedProperties("Allowed Properties", "Properties for which we will allow updates");
-core::Property UpdatePolicyControllerService::DisallowedProperties("Disallowed Properties", "Properties for which we will not allow updates");
-core::Property UpdatePolicyControllerService::PersistUpdates("Persist Updates", "Property that dictates whether updates should persist after a restart");
+core::Property UpdatePolicyControllerService::AllowAllProperties(
+    core::PropertyBuilder::createProperty("Allow All Properties")->withDescription("Allows all properties, which are also not disallowed, to be updated")->withDefaultValue<bool>(
+        false)->build());
+
+core::Property UpdatePolicyControllerService::AllowedProperties(
+    core::PropertyBuilder::createProperty("Allowed Properties")->withDescription("Properties for which we will allow updates")->isRequired(false)->build());
+
+core::Property UpdatePolicyControllerService::DisallowedProperties(
+    core::PropertyBuilder::createProperty("Disallowed Properties")->withDescription("Properties for which we will not allow updates")->isRequired(false)->build());
+
+core::Property UpdatePolicyControllerService::PersistUpdates(
+    core::PropertyBuilder::createProperty("Persist Updates")->withDescription("Property that dictates whether updates should persist after a restart")->isRequired(false)->withDefaultValue<bool>(false)
+        ->build());
 
 void UpdatePolicyControllerService::initialize() {
   std::set<core::Property> supportedProperties;
diff --git a/libminifi/src/core/ConfigurableComponent.cpp b/libminifi/src/core/ConfigurableComponent.cpp
index 3923cfca..ed54b4f4 100644
--- a/libminifi/src/core/ConfigurableComponent.cpp
+++ b/libminifi/src/core/ConfigurableComponent.cpp
@@ -57,25 +57,6 @@ bool ConfigurableComponent::getProperty(const std::string &name, Property &prop)
   }
 }
 
-/**
- * Get property using the provided name.
- * @param name property name.
- * @param value value passed in by reference
- * @return result of getting property.
- */
-bool ConfigurableComponent::getProperty(const std::string name, std::string &value) const {
-  std::lock_guard<std::mutex> lock(configuration_mutex_);
-
-  auto &&it = properties_.find(name);
-  if (it != properties_.end()) {
-    Property item = it->second;
-    value = item.getValue();
-    logger_->log_debug("Component %s property name %s value %s", name, item.getName(), value);
-    return true;
-  } else {
-    return false;
-  }
-}
 /**
  * Sets the property using the provided name
  * @param property name
@@ -95,6 +76,7 @@ bool ConfigurableComponent::setProperty(const std::string name, std::string valu
     logger_->log_debug("Component %s property name %s value %s", name, new_property.getName(), value);
     return true;
   } else {
+    logger_->log_debug("Component %s cannot be set to %s", name, value);
     return false;
   }
 }
@@ -138,7 +120,7 @@ bool ConfigurableComponent::setProperty(Property &prop, std::string value) {
     new_property.setValue(value);
     properties_[new_property.getName()] = new_property;
     onPropertyModified(orig_property, new_property);
-    logger_->log_debug("property name %s value %s", prop.getName(), new_property.getName(), value);
+    logger_->log_debug("property name %s value %s and new value is %s", prop.getName(), value, new_property.getValue().to_string());
     return true;
   } else {
     Property new_property(prop);
@@ -149,6 +131,31 @@ bool ConfigurableComponent::setProperty(Property &prop, std::string value) {
   }
 }
 
+bool ConfigurableComponent::setProperty(Property &prop, PropertyValue &value) {
+  std::lock_guard<std::mutex> lock(configuration_mutex_);
+    auto it = properties_.find(prop.getName());
+
+    if (it != properties_.end()) {
+      Property &orig_property = it->second;
+      Property new_property = orig_property;
+      new_property.setValue(value);
+      properties_[new_property.getName()] = new_property;
+      onPropertyModified(orig_property, new_property);
+      logger_->log_debug("property name %s value %s and new value is %s", prop.getName(), new_property.getName(), value, new_property.getValue().to_string());
+      return true;
+    } else {
+      if (supportsDynamicProperties()) {
+        Property new_property(prop);
+        new_property.setValue(value);
+        properties_.insert(std::pair<std::string, Property>(prop.getName(), new_property));
+        onPropertyModified({}, new_property);
+        return true;
+      } else {
+        return false;
+      }
+    }
+}
+
 /**
  * Sets supported properties for the ConfigurableComponent
  * @param supported properties
@@ -174,7 +181,7 @@ bool ConfigurableComponent::getDynamicProperty(const std::string name, std::stri
   auto &&it = dynamic_properties_.find(name);
   if (it != dynamic_properties_.end()) {
     Property item = it->second;
-    value = item.getValue();
+    value = item.getValue().to_string();
     logger_->log_debug("Component %s dynamic property name %s value %s", name, item.getName(), value);
     return true;
   } else {
diff --git a/libminifi/src/core/FlowConfiguration.cpp b/libminifi/src/core/FlowConfiguration.cpp
index aea6d624..1d1f5300 100644
--- a/libminifi/src/core/FlowConfiguration.cpp
+++ b/libminifi/src/core/FlowConfiguration.cpp
@@ -36,7 +36,7 @@ static_initializers &get_static_functions() {
 FlowConfiguration::~FlowConfiguration() {
 }
 
-std::shared_ptr<core::Processor> FlowConfiguration::createProcessor(std::string name, utils::Identifier &  uuid) {
+std::shared_ptr<core::Processor> FlowConfiguration::createProcessor(std::string name, utils::Identifier & uuid) {
   auto ptr = core::ClassLoader::getDefaultClassLoader().instantiate(name, uuid);
   if (nullptr == ptr) {
     logger_->log_error("No Processor defined for %s", name);
@@ -62,7 +62,12 @@ std::shared_ptr<core::Processor> FlowConfiguration::createProvenanceReportTask()
 }
 
 std::unique_ptr<core::ProcessGroup> FlowConfiguration::updateFromPayload(const std::string &source, const std::string &yamlConfigPayload) {
-  if (!source.empty()) {
+  auto old_services = controller_services_;
+  auto old_provider = service_provider_;
+  controller_services_ = std::make_shared<core::controller::ControllerServiceMap>();
+  service_provider_ = std::make_shared<core::controller::StandardControllerServiceProvider>(controller_services_, nullptr, configuration_);
+  auto payload = getRootFromPayload(yamlConfigPayload);
+  if (!source.empty() && payload != nullptr) {
     std::string host, protocol, path, query, url = source;
     int port;
     utils::parse_url(&url, &host, &port, &protocol, &path, &query);
@@ -86,23 +91,26 @@ std::unique_ptr<core::ProcessGroup> FlowConfiguration::updateFromPayload(const s
       }
     }
     flow_version_->setFlowVersion(url, bucket_id, flow_id);
+  } else {
+    controller_services_ = old_services;
+    service_provider_ = old_provider;
   }
-  return getRootFromPayload(yamlConfigPayload);
+  return payload;
 }
 
-std::unique_ptr<core::ProcessGroup> FlowConfiguration::createRootProcessGroup(std::string name, utils::Identifier &  uuid, int version) {
+std::unique_ptr<core::ProcessGroup> FlowConfiguration::createRootProcessGroup(std::string name, utils::Identifier & uuid, int version) {
   return std::unique_ptr<core::ProcessGroup>(new core::ProcessGroup(core::ROOT_PROCESS_GROUP, name, uuid, version));
 }
 
-std::unique_ptr<core::ProcessGroup> FlowConfiguration::createRemoteProcessGroup(std::string name, utils::Identifier &  uuid) {
+std::unique_ptr<core::ProcessGroup> FlowConfiguration::createRemoteProcessGroup(std::string name, utils::Identifier & uuid) {
   return std::unique_ptr<core::ProcessGroup>(new core::ProcessGroup(core::REMOTE_PROCESS_GROUP, name, uuid));
 }
 
-std::shared_ptr<minifi::Connection> FlowConfiguration::createConnection(std::string name, utils::Identifier &  uuid) {
+std::shared_ptr<minifi::Connection> FlowConfiguration::createConnection(std::string name, utils::Identifier & uuid) {
   return std::make_shared<minifi::Connection>(flow_file_repo_, content_repo_, name, uuid);
 }
 
-std::shared_ptr<core::controller::ControllerServiceNode> FlowConfiguration::createControllerService(const std::string &class_name, const std::string &name, utils::Identifier &  uuid) {
+std::shared_ptr<core::controller::ControllerServiceNode> FlowConfiguration::createControllerService(const std::string &class_name, const std::string &name, utils::Identifier & uuid) {
   std::shared_ptr<core::controller::ControllerServiceNode> controllerServicesNode = service_provider_->createControllerService(class_name, name, true);
   if (nullptr != controllerServicesNode)
     controllerServicesNode->setUUID(uuid);
diff --git a/libminifi/src/core/Property.cpp b/libminifi/src/core/Property.cpp
index f3205d63..e58833e9 100644
--- a/libminifi/src/core/Property.cpp
+++ b/libminifi/src/core/Property.cpp
@@ -39,11 +39,11 @@ std::vector<std::string> Property::getAllowedTypes() const {
   return types_;
 }
 
-std::string Property::getValue() const {
+const PropertyValue &Property::getValue() const {
   if (!values_.empty())
     return values_.front();
   else
-    return "";
+    return default_value_;
 }
 
 bool Property::getRequired() const {
@@ -58,45 +58,32 @@ std::string Property::getValidRegex() const {
   return valid_regex_;
 }
 
-std::vector<std::string> &Property::getValues() {
-  return values_;
+std::shared_ptr<PropertyValidator> Property::getValidator() const {
+  return validator_;
 }
 
-void Property::setValue(std::string value) {
-  if (!is_collection_) {
-    values_.clear();
-    values_.push_back(std::move(value));
-  } else {
-    values_.push_back(std::move(value));
+std::vector<std::string> Property::getValues() {
+  std::vector<std::string> values;
+  for (const auto &v : values_) {
+    values.push_back(v.to_string());
   }
+  return values;
 }
 
 void Property::setSupportsExpressionLanguage(bool supportEl) {
   supports_el_ = supportEl;
 }
 
-void Property::addValue(std::string value) {
-  values_.push_back(std::move(value));
+void Property::addValue(const std::string &value) {
+  PropertyValue vn;
+  vn = value;
+  values_.push_back(std::move(vn));
 }
 
 bool Property::operator<(const Property &right) const {
   return name_ < right.name_;
 }
 
-const Property &Property::operator=(const Property &other) {
-  name_ = other.name_;
-  display_name_ = other.display_name_;
-  types_ = other.types_;
-  values_ = other.values_;
-  description_ = other.description_;
-  is_collection_ = other.is_collection_;
-  is_required_ = other.is_required_;
-  valid_regex_ = other.valid_regex_;
-  dependent_properties_ = other.dependent_properties_;
-  exclusive_of_properties_ = other.exclusive_of_properties_;
-  supports_el_ = other.supports_el_;
-  return *this;
-}
 
 std::vector<std::string> Property::getDependentProperties() const {
   return dependent_properties_;
diff --git a/libminifi/src/core/PropertyValidation.cpp b/libminifi/src/core/PropertyValidation.cpp
new file mode 100644
index 00000000..d92b0736
--- /dev/null
+++ b/libminifi/src/core/PropertyValidation.cpp
@@ -0,0 +1,40 @@
+/**
+ *
+ * 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 "core/PropertyValidation.h"
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+std::shared_ptr<PropertyValidator> StandardValidators::VALID = std::make_shared<AlwaysValid>(true, "VALID");
+StandardValidators::StandardValidators() {
+  INVALID = std::make_shared<AlwaysValid>(false, "INVALID");
+  INTEGER_VALIDATOR = std::make_shared<IntegerValidator>("INTEGER_VALIDATOR");
+  LONG_VALIDATOR = std::make_shared<LongValidator>("LONG_VALIDATOR");
+  UNSIGNED_LONG_VALIDATOR = std::make_shared<UnsignedLongValidator>("LONG_VALIDATOR");
+  DATA_SIZE_VALIDATOR = std::make_shared<DataSizeValidator>("DATA_SIZE_VALIDATOR");
+  TIME_PERIOD_VALIDATOR = std::make_shared<TimePeriodValidator>("TIME_PERIOD_VALIDATOR");
+  BOOLEAN_VALIDATOR = std::make_shared<BooleanValidator>("BOOLEAN_VALIDATOR");
+}
+} /* namespace core */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
diff --git a/libminifi/src/core/TypedValues.cpp b/libminifi/src/core/TypedValues.cpp
new file mode 100644
index 00000000..93a43b8f
--- /dev/null
+++ b/libminifi/src/core/TypedValues.cpp
@@ -0,0 +1,34 @@
+/**
+ *
+ * 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 "core/Property.h"
+#include "core/TypedValues.h"
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+const  std::type_index DataSizeValue::type_id = typeid(uint64_t);
+const  std::type_index TimePeriodValue::type_id = typeid(uint64_t);
+
+} /* namespace core */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
diff --git a/libminifi/src/core/state/Value.cpp b/libminifi/src/core/state/Value.cpp
new file mode 100644
index 00000000..4c43c3e2
--- /dev/null
+++ b/libminifi/src/core/state/Value.cpp
@@ -0,0 +1,42 @@
+/**
+ *
+ * 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 "core/state/Value.h"
+#include <utility>
+#include <string>
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace state {
+namespace response {
+
+const std::type_index Value::UINT64_TYPE = std::type_index(typeid(uint64_t));
+const std::type_index Value::INT64_TYPE = std::type_index(typeid(int64_t));
+const std::type_index Value::INT_TYPE = std::type_index(typeid(int));
+const std::type_index Value::BOOL_TYPE = std::type_index(typeid(bool));
+const std::type_index Value::STRING_TYPE = std::type_index(typeid(std::string));
+
+} /* namespace response */
+} /* namespace state */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
+
diff --git a/libminifi/src/core/yaml/YamlConfiguration.cpp b/libminifi/src/core/yaml/YamlConfiguration.cpp
index 5458597a..ab1f0865 100644
--- a/libminifi/src/core/yaml/YamlConfiguration.cpp
+++ b/libminifi/src/core/yaml/YamlConfiguration.cpp
@@ -21,7 +21,7 @@
 #include <set>
 
 #include "core/yaml/YamlConfiguration.h"
-
+#include "core/state/Value.h"
 #ifdef YAML_CONFIGURATION_USE_REGEX
 #include <regex>
 #endif  // YAML_CONFIGURATION_USE_REGEX
@@ -119,6 +119,7 @@ void YamlConfiguration::parseProcessorNodeYaml(YAML::Node processorsNode, core::
           logger_->log_error("Could not create a processor %s with id %s", procCfg.name, procCfg.id);
           throw std::invalid_argument("Could not create processor " + procCfg.name);
         }
+
         processor->setName(procCfg.name);
 
         processor->setFlowIdentifier(flow_version_->getFlowIdentifier());
@@ -451,12 +452,26 @@ void YamlConfiguration::parseControllerServices(YAML::Node *controllerServicesNo
           CONFIG_YAML_CONTROLLER_SERVICES_KEY);
           checkRequiredField(&controllerServiceNode, "id",
           CONFIG_YAML_CONTROLLER_SERVICES_KEY);
-          checkRequiredField(&controllerServiceNode, "class",
-          CONFIG_YAML_CONTROLLER_SERVICES_KEY);
+          std::string type = "";
+
+          try {
+            checkRequiredField(&controllerServiceNode, "class", CONFIG_YAML_CONTROLLER_SERVICES_KEY);
+            type = controllerServiceNode["class"].as<std::string>();
+          } catch (const std::invalid_argument &e) {
+            checkRequiredField(&controllerServiceNode, "type", CONFIG_YAML_CONTROLLER_SERVICES_KEY);
+            type = controllerServiceNode["type"].as<std::string>();
+            logger_->log_debug("Using type %s for controller service node", type);
+          }
+
+          auto lastOfIdx = type.find_last_of(".");
+          if (lastOfIdx != std::string::npos) {
+            lastOfIdx++;  // if a value is found, increment to move beyond the .
+            int nameLength = type.length() - lastOfIdx;
+            type = type.substr(lastOfIdx, nameLength);
+          }
 
           auto name = controllerServiceNode["name"].as<std::string>();
           auto id = controllerServiceNode["id"].as<std::string>();
-          auto type = controllerServiceNode["class"].as<std::string>();
 
           utils::Identifier uuid;
           uuid = id;
@@ -472,6 +487,8 @@ void YamlConfiguration::parseControllerServices(YAML::Node *controllerServicesNo
               parsePropertiesNodeYaml(&propertiesNode, std::static_pointer_cast<core::ConfigurableComponent>(controller_service_node->getControllerServiceImplementation()), name,
               CONFIG_YAML_CONTROLLER_SERVICES_KEY);
             }
+          } else {
+            logger_->log_debug("Could not locate %s", type);
           }
           controller_services_->put(id, controller_service_node);
           controller_services_->put(name, controller_service_node);
@@ -704,9 +721,11 @@ void YamlConfiguration::parsePortYaml(YAML::Node *portNode, core::ProcessGroup *
 void YamlConfiguration::parsePropertiesNodeYaml(YAML::Node *propertiesNode, std::shared_ptr<core::ConfigurableComponent> processor, const std::string &component_name,
                                                 const std::string &yaml_section) {
   // Treat generically as a YAML node so we can perform inspection on entries to ensure they are populated
+  logger_->log_trace("Entered %s", component_name);
   for (YAML::const_iterator propsIter = propertiesNode->begin(); propsIter != propertiesNode->end(); ++propsIter) {
     std::string propertyName = propsIter->first.as<std::string>();
     YAML::Node propertyValueNode = propsIter->second;
+    logger_->log_trace("Encountered %s", propertyName);
     if (!propertyValueNode.IsNull() && propertyValueNode.IsDefined()) {
       if (propertyValueNode.IsSequence()) {
         for (auto iter : propertyValueNode) {
@@ -732,8 +751,56 @@ void YamlConfiguration::parsePropertiesNodeYaml(YAML::Node *propertiesNode, std:
           }
         }
       } else {
+        core::Property myProp;
+        processor->getProperty(propertyName, myProp);
+        PropertyValue defaultValue;
+        defaultValue = myProp.getDefaultValue();
+        auto defaultType = defaultValue.getTypeInfo();
+        PropertyValue coercedValue = defaultValue;
+
+        // coerce the types. upon failure we will either exit or use the default value.
+        // we do this here ( in addition to the PropertyValue class ) to get the earliest
+        // possible YAML failure.
+        try {
+          if (defaultType == typeid(std::string)) {
+            auto typedValue = propertyValueNode.as<std::string>();
+            coercedValue = typedValue;
+          } else if (defaultType == typeid(int64_t)) {
+            auto typedValue = propertyValueNode.as<int64_t>();
+            coercedValue = typedValue;
+          } else if (defaultType == typeid(uint64_t)) {
+            try {
+              auto typedValue = propertyValueNode.as<uint64_t>();
+              coercedValue = typedValue;
+            } catch (...) {
+              auto typedValue = propertyValueNode.as<std::string>();
+              coercedValue = typedValue;
+            }
+          } else if (defaultType == typeid(int)) {
+            auto typedValue = propertyValueNode.as<int>();
+            coercedValue = typedValue;
+          } else if (defaultType == typeid(bool)) {
+            auto typedValue = propertyValueNode.as<bool>();
+            coercedValue = typedValue;
+          } else {
+            auto typedValue = propertyValueNode.as<std::string>();
+            coercedValue = typedValue;
+          }
+        } catch (...) {
+          std::string eof;
+          bool exit_on_failure = false;
+          if (configuration_->get(Configure::nifi_flow_configuration_file_exit_failure, eof)) {
+            utils::StringUtils::StringToBool(eof, exit_on_failure);
+          }
+          logger_->log_error("Invalid conversion for field %s. Value %s", myProp.getName(), propertyValueNode.as<std::string>());
+          if (exit_on_failure) {
+            std::cerr << "Invalid conversion for " << myProp.getName() << " to " << defaultType.name() << std::endl;
+          } else {
+            coercedValue = defaultValue;
+          }
+        }
         std::string rawValueString = propertyValueNode.as<std::string>();
-        if (!processor->setProperty(propertyName, rawValueString)) {
+        if (!processor->setProperty(myProp, coercedValue)) {
           std::shared_ptr<core::Connectable> proc = std::dynamic_pointer_cast<core::Connectable>(processor);
           if (proc != 0) {
             logger_->log_warn("Received property %s with value %s but is not one of the properties for %s. "
@@ -759,10 +826,14 @@ void YamlConfiguration::validateComponentProperties(const std::shared_ptr<Config
   // Validate required properties
   for (const auto &prop_pair : component_properties) {
     if (prop_pair.second.getRequired()) {
-      if (prop_pair.second.getValue().empty()) {
+      if (prop_pair.second.getValue().to_string().empty()) {
         std::stringstream reason;
         reason << "required property '" << prop_pair.second.getName() << "' is not set";
         raiseComponentError(component_name, yaml_section, reason.str());
+      } else if (!prop_pair.second.getValue().validate(prop_pair.first).valid()) {
+        std::stringstream reason;
+        reason << "Property '" << prop_pair.second.getName() << "' is not valid";
+        raiseComponentError(component_name, yaml_section, reason.str());
       }
     }
   }
@@ -771,12 +842,12 @@ void YamlConfiguration::validateComponentProperties(const std::shared_ptr<Config
   for (const auto &prop_pair : component_properties) {
     const auto &dep_props = prop_pair.second.getDependentProperties();
 
-    if (prop_pair.second.getValue().empty()) {
+    if (prop_pair.second.getValue().to_string().empty()) {
       continue;
     }
 
     for (const auto &dep_prop_key : dep_props) {
-      if (component_properties.at(dep_prop_key).getValue().empty()) {
+      if (component_properties.at(dep_prop_key).getValue().to_string().empty()) {
         std::string reason("property '");
         reason.append(prop_pair.second.getName());
         reason.append("' depends on property '");
@@ -798,7 +869,7 @@ void YamlConfiguration::validateComponentProperties(const std::shared_ptr<Config
 
     for (const auto &excl_pair : excl_props) {
       std::regex excl_expr(excl_pair.second);
-      if (std::regex_match(component_properties.at(excl_pair.first).getValue(), excl_expr)) {
+      if (std::regex_match(component_properties.at(excl_pair.first).getValue().to_string(), excl_expr)) {
         std::string reason("property '");
         reason.append(prop_pair.second.getName());
         reason.append("' is exclusive of property '");
@@ -817,7 +888,7 @@ void YamlConfiguration::validateComponentProperties(const std::shared_ptr<Config
 
     if (!prop_regex_str.empty()) {
       std::regex prop_regex(prop_regex_str);
-      if (!std::regex_match(prop_pair.second.getValue(), prop_regex)) {
+      if (!std::regex_match(prop_pair.second.getValue().to_string(), prop_regex)) {
         std::stringstream reason;
         reason << "property '" << prop_pair.second.getName() << "' does not match validation pattern '" << prop_regex_str << "'";
         raiseComponentError(component_name, yaml_section, reason.str());
diff --git a/libminifi/src/processors/ExecuteProcess.cpp b/libminifi/src/processors/ExecuteProcess.cpp
index 6f43e8a1..3b0e0756 100644
--- a/libminifi/src/processors/ExecuteProcess.cpp
+++ b/libminifi/src/processors/ExecuteProcess.cpp
@@ -26,6 +26,7 @@
 #include "core/ProcessSession.h"
 #include "utils/StringUtils.h"
 #include "utils/TimeUtil.h"
+#include "core/TypedValues.h"
 
 #if defined(__clang__)
 #pragma clang diagnostic push
@@ -53,10 +54,15 @@ core::Property ExecuteProcess::CommandArguments(
 core::Property ExecuteProcess::WorkingDir(
     core::PropertyBuilder::createProperty("Working Directory")->withDescription("The directory to use as the current working directory when executing the command")->supportsExpressionLanguage(true)
         ->withDefaultValue("")->build());
-core::Property ExecuteProcess::BatchDuration("Batch Duration", "If the process is expected to be long-running and produce textual output, a "
-                                             "batch duration can be specified.",
-                                             "0");
-core::Property ExecuteProcess::RedirectErrorStream("Redirect Error Stream", "If true will redirect any error stream output of the process to the output stream.", "false");
+
+core::Property ExecuteProcess::BatchDuration(
+    core::PropertyBuilder::createProperty("Batch Duration")->withDescription("If the process is expected to be long-running and produce textual output, a "
+                                                                             "batch duration can be specified.")->withDefaultValue<core::TimePeriodValue>("0 sec")->build());
+
+core::Property ExecuteProcess::RedirectErrorStream(
+    core::PropertyBuilder::createProperty("Redirect Error Stream")->withDescription("If true will redirect any error stream output of the process to the output stream.")->withDefaultValue<bool>(false)
+        ->build());
+
 core::Relationship ExecuteProcess::Success("success", "All created FlowFiles are routed to this relationship.");
 
 void ExecuteProcess::initialize() {
diff --git a/libminifi/src/processors/ExtractText.cpp b/libminifi/src/processors/ExtractText.cpp
index 264838ea..fcd9eea5 100644
--- a/libminifi/src/processors/ExtractText.cpp
+++ b/libminifi/src/processors/ExtractText.cpp
@@ -38,8 +38,13 @@ namespace processors {
 
 #define MAX_BUFFER_SIZE 4096
 
-core::Property ExtractText::Attribute("Attribute", "Attribute to set from content", "");
-core::Property ExtractText::SizeLimit("Size Limit", "Maximum number of bytes to read into the attribute. 0 for no limit. Default is 2MB.");
+core::Property ExtractText::Attribute(core::PropertyBuilder::createProperty("Attribute")->withDescription("Attribute to set from content")->build());
+
+// despite there being a size value, ExtractText was initially built with a numeric for this property
+core::Property ExtractText::SizeLimit(
+    core::PropertyBuilder::createProperty("Size Limit")->withDescription("Maximum number of bytes to read into the attribute. 0 for no limit. Default is 2MB.")->withDefaultValue<uint32_t>(
+        DEFAULT_SIZE_LIMIT)->build());
+
 core::Relationship ExtractText::Success("success", "success operational on the flow record");
 
 void ExtractText::initialize() {
diff --git a/libminifi/src/processors/GenerateFlowFile.cpp b/libminifi/src/processors/GenerateFlowFile.cpp
index 223346f9..a1b62ecb 100644
--- a/libminifi/src/processors/GenerateFlowFile.cpp
+++ b/libminifi/src/processors/GenerateFlowFile.cpp
@@ -35,6 +35,8 @@
 #include "utils/StringUtils.h"
 #include "core/ProcessContext.h"
 #include "core/ProcessSession.h"
+#include "core/PropertyValidation.h"
+#include "core/TypedValues.h"
 
 namespace org {
 namespace apache {
@@ -43,10 +45,20 @@ namespace minifi {
 namespace processors {
 const char *GenerateFlowFile::DATA_FORMAT_BINARY = "Binary";
 const char *GenerateFlowFile::DATA_FORMAT_TEXT = "Text";
-core::Property GenerateFlowFile::FileSize("File Size", "The size of the file that will be used", "1 kB");
-core::Property GenerateFlowFile::BatchSize("Batch Size", "The number of FlowFiles to be transferred in each invocation", "1");
-core::Property GenerateFlowFile::DataFormat("Data Format", "Specifies whether the data should be Text or Binary", GenerateFlowFile::DATA_FORMAT_BINARY);
-core::Property GenerateFlowFile::UniqueFlowFiles("Unique FlowFiles", "If true, each FlowFile that is generated will be unique. If false, a random value will be generated and all FlowFiles", "true");
+core::Property GenerateFlowFile::FileSize(
+    core::PropertyBuilder::createProperty("File Size")->withDescription("The size of the file that will be used")->isRequired(false)->withDefaultValue<core::DataSizeValue>("1 kB")->build());
+
+core::Property GenerateFlowFile::BatchSize(
+    core::PropertyBuilder::createProperty("Batch Size")->withDescription("The number of FlowFiles to be transferred in each invocation")->isRequired(false)->withDefaultValue<int>(1)->build());
+
+core::Property GenerateFlowFile::DataFormat(
+    core::PropertyBuilder::createProperty("Data Format")->withDescription("Specifies whether the data should be Text or Binary")->isRequired(false)->withAllowableValue<std::string>("Text")
+        ->withAllowableValue("Binary")->withDefaultValue("Binary")->build());
+
+core::Property GenerateFlowFile::UniqueFlowFiles(
+    core::PropertyBuilder::createProperty("Unique FlowFiles")->withDescription("If true, each FlowFile that is generated will be unique. If false, a random value will be generated and all FlowFiles")
+        ->isRequired(false)->withDefaultValue<bool>(true)->build());
+
 core::Relationship GenerateFlowFile::Success("success", "success operational on the flow record");
 const unsigned int TEXT_LEN = 90;
 static const char TEXT_CHARS[TEXT_LEN + 1] = "abcdefghijklmnopqrstuvwxyzABCDEFGHIJKLMNOPQRSTUVWXYZ1234567890!@#$%^&*()-_=+/?.,';:\"?<>\n\t ";
@@ -66,23 +78,24 @@ void GenerateFlowFile::initialize() {
 }
 
 void GenerateFlowFile::onTrigger(core::ProcessContext *context, core::ProcessSession *session) {
-  int64_t batchSize = 1;
+  uint64_t batchSize = 1;
   bool uniqueFlowFile = true;
-  int64_t fileSize = 1024;
+  uint64_t fileSize = 1024;
   bool textData = false;
 
   std::string value;
-  if (context->getProperty(FileSize.getName(), value)) {
-    core::Property::StringToInt(value, fileSize);
+  if (context->getProperty(FileSize.getName(), fileSize)) {
+    logger_->log_trace("File size is configured to be %d", fileSize);
   }
-  if (context->getProperty(BatchSize.getName(), value)) {
-    core::Property::StringToInt(value, batchSize);
+
+  if (context->getProperty(BatchSize.getName(), batchSize)) {
+    logger_->log_trace("Batch size is configured to be %d", batchSize);
   }
   if (context->getProperty(DataFormat.getName(), value)) {
     textData = (value == GenerateFlowFile::DATA_FORMAT_TEXT);
   }
-  if (context->getProperty(UniqueFlowFiles.getName(), value)) {
-    org::apache::nifi::minifi::utils::StringUtils::StringToBool(value, uniqueFlowFile);
+  if (context->getProperty(UniqueFlowFiles.getName(), uniqueFlowFile)) {
+    logger_->log_trace("Unique Flow files is configured to be %i", uniqueFlowFile);
   }
 
   if (uniqueFlowFile) {
@@ -94,18 +107,18 @@ void GenerateFlowFile::onTrigger(core::ProcessContext *context, core::ProcessSes
     GenerateFlowFile::WriteCallback callback(data, dataSize);
     char *current = data;
     if (textData) {
-      for (int i = 0; i < fileSize; i++) {
+      for (uint64_t i = 0; i < fileSize; i++) {
         int randValue = random();
         data[i] = TEXT_CHARS[randValue % TEXT_LEN];
       }
     } else {
-      for (int i = 0; i < fileSize; i += sizeof(int)) {
+      for (uint64_t i = 0; i < fileSize; i += sizeof(int)) {
         int randValue = random();
         *(reinterpret_cast<int*>(current)) = randValue;
         current += sizeof(int);
       }
     }
-    for (int i = 0; i < batchSize; i++) {
+    for (uint64_t i = 0; i < batchSize; i++) {
       // For each batch
       std::shared_ptr<FlowFileRecord> flowFile = std::static_pointer_cast<FlowFileRecord>(session->create());
       if (!flowFile)
@@ -122,12 +135,12 @@ void GenerateFlowFile::onTrigger(core::ProcessContext *context, core::ProcessSes
       _dataSize = fileSize;
       char *current = _data;
       if (textData) {
-        for (int i = 0; i < fileSize; i++) {
+        for (uint64_t i = 0; i < fileSize; i++) {
           int randValue = random();
           _data[i] = TEXT_CHARS[randValue % TEXT_LEN];
         }
       } else {
-        for (int i = 0; i < fileSize; i += sizeof(int)) {
+        for (uint64_t i = 0; i < fileSize; i += sizeof(int)) {
           int randValue = random();
           *(reinterpret_cast<int*>(current)) = randValue;
           current += sizeof(int);
@@ -135,7 +148,7 @@ void GenerateFlowFile::onTrigger(core::ProcessContext *context, core::ProcessSes
       }
     }
     GenerateFlowFile::WriteCallback callback(_data, _dataSize);
-    for (int i = 0; i < batchSize; i++) {
+    for (uint64_t i = 0; i < batchSize; i++) {
       // For each batch
       std::shared_ptr<FlowFileRecord> flowFile = std::static_pointer_cast<FlowFileRecord>(session->create());
       if (!flowFile)
diff --git a/libminifi/src/processors/GetFile.cpp b/libminifi/src/processors/GetFile.cpp
index 89e50fce..3fbd41f1 100644
--- a/libminifi/src/processors/GetFile.cpp
+++ b/libminifi/src/processors/GetFile.cpp
@@ -39,6 +39,7 @@
 #include "utils/TimeUtil.h"
 #include "core/ProcessContext.h"
 #include "core/ProcessSession.h"
+#include "core/TypedValues.h"
 
 #ifndef S_ISDIR
 #define S_ISDIR(mode)  (((mode) & S_IFMT) == S_IFDIR)
@@ -53,23 +54,52 @@ namespace nifi {
 namespace minifi {
 namespace processors {
 
-core::Property GetFile::BatchSize("Batch Size", "The maximum number of files to pull in each iteration", "10");
+core::Property GetFile::BatchSize(
+    core::PropertyBuilder::createProperty("Batch Size")->withDescription("The maximum number of files to pull in each iteration")->withDefaultValue<uint32_t>(10)
+        ->build());
+
 core::Property GetFile::Directory(
     core::PropertyBuilder::createProperty("Input Directory")->withDescription("The input directory from which to pull files")->isRequired(true)->supportsExpressionLanguage(true)->withDefaultValue(".")
         ->build());
-core::Property GetFile::IgnoreHiddenFile("Ignore Hidden Files", "Indicates whether or not hidden files should be ignored", "true");
-core::Property GetFile::KeepSourceFile("Keep Source File", "If true, the file is not deleted after it has been copied to the Content Repository", "false");
-core::Property GetFile::MaxAge("Maximum File Age", "The minimum age that a file must be in order to be pulled;"
-                               " any file younger than this amount of time (according to last modification date) will be ignored",
-                               "0 sec");
-core::Property GetFile::MinAge("Minimum File Age", "The maximum age that a file must be in order to be pulled; any file"
-                               "older than this amount of time (according to last modification date) will be ignored",
-                               "0 sec");
-core::Property GetFile::MaxSize("Maximum File Size", "The maximum size that a file can be in order to be pulled", "0 B");
-core::Property GetFile::MinSize("Minimum File Size", "The minimum size that a file must be in order to be pulled", "0 B");
-core::Property GetFile::PollInterval("Polling Interval", "Indicates how long to wait before performing a directory listing", "0 sec");
-core::Property GetFile::Recurse("Recurse Subdirectories", "Indicates whether or not to pull files from subdirectories", "true");
-core::Property GetFile::FileFilter("File Filter", "Only files whose names match the given regular expression will be picked up", "[^\\.].*");
+
+core::Property GetFile::IgnoreHiddenFile(
+    core::PropertyBuilder::createProperty("Ignore Hidden Files")->withDescription("Indicates whether or not hidden files should be ignored")->withDefaultValue<bool>(true)
+        ->build());
+
+core::Property GetFile::KeepSourceFile(
+    core::PropertyBuilder::createProperty("Keep Source File")->withDescription("If true, the file is not deleted after it has been copied to the Content Repository")->withDefaultValue<bool>(false)
+        ->build());
+
+core::Property GetFile::MaxAge(
+    core::PropertyBuilder::createProperty("Maximum File Age")->withDescription("The maximum age that a file must be in order to be pulled;"
+                               " any file older than this amount of time (according to last modification date) will be ignored")->withDefaultValue<core::TimePeriodValue>("0 sec")
+        ->build());
+
+core::Property GetFile::MinAge(
+    core::PropertyBuilder::createProperty("Minimum File Age")->withDescription("The minimum age that a file must be in order to be pulled;"
+                               " any file younger than this amount of time (according to last modification date) will be ignored")->withDefaultValue<core::TimePeriodValue>("0 sec")
+        ->build());
+
+core::Property GetFile::MaxSize(
+    core::PropertyBuilder::createProperty("Minimum File Size")->withDescription("The maximum size that a file can be in order to be pulled")->withDefaultValue<core::DataSizeValue>("0 B")
+        ->build());
+
+core::Property GetFile::MinSize(
+    core::PropertyBuilder::createProperty("Minimum File Size")->withDescription("The minimum size that a file can be in order to be pulled")->withDefaultValue<core::DataSizeValue>("0 B")
+        ->build());
+
+core::Property GetFile::PollInterval(
+    core::PropertyBuilder::createProperty("Polling Interval")->withDescription("Indicates how long to wait before performing a directory listing")->withDefaultValue<core::TimePeriodValue>("0 sec")
+        ->build());
+
+core::Property GetFile::Recurse(
+    core::PropertyBuilder::createProperty("Recurse Subdirectories")->withDescription("Indicates whether or not to pull files from subdirectories")->withDefaultValue<bool>(true)
+        ->build());
+
+core::Property GetFile::FileFilter(
+    core::PropertyBuilder::createProperty("File Filter")->withDescription("Only files whose names match the given regular expression will be picked up")->withDefaultValue("[^\\.].*")
+        ->build());
+
 core::Relationship GetFile::Success("success", "All files are routed to success");
 
 void GetFile::initialize() {
@@ -105,30 +135,18 @@ void GetFile::onSchedule(core::ProcessContext *context, core::ProcessSessionFact
     org::apache::nifi::minifi::utils::StringUtils::StringToBool(value, request_.keepSourceFile);
   }
 
-  if (context->getProperty(MaxAge.getName(), value)) {
-    core::TimeUnit unit;
-    if (core::Property::StringToTime(value, request_.maxAge, unit) && core::Property::ConvertTimeUnitToMS(request_.maxAge, unit, request_.maxAge)) {
-      logger_->log_debug("successfully applied _maxAge");
-    }
-  }
-  if (context->getProperty(MinAge.getName(), value)) {
-    core::TimeUnit unit;
-    if (core::Property::StringToTime(value, request_.minAge, unit) && core::Property::ConvertTimeUnitToMS(request_.minAge, unit, request_.minAge)) {
-      logger_->log_debug("successfully applied _minAge");
-    }
-  }
+  context->getProperty(MaxAge.getName(), request_.maxAge);
+  context->getProperty(MinAge.getName(), request_.minAge);
+
   if (context->getProperty(MaxSize.getName(), value)) {
     core::Property::StringToInt(value, request_.maxSize);
   }
   if (context->getProperty(MinSize.getName(), value)) {
     core::Property::StringToInt(value, request_.minSize);
   }
-  if (context->getProperty(PollInterval.getName(), value)) {
-    core::TimeUnit unit;
-    if (core::Property::StringToTime(value, request_.pollInterval, unit) && core::Property::ConvertTimeUnitToMS(request_.pollInterval, unit, request_.pollInterval)) {
-      logger_->log_debug("successfully applied _pollInterval");
-    }
-  }
+
+  context->getProperty(PollInterval.getName(), request_.pollInterval);
+
   if (context->getProperty(Recurse.getName(), value)) {
     org::apache::nifi::minifi::utils::StringUtils::StringToBool(value, request_.recursive);
   }
diff --git a/libminifi/src/processors/GetTCP.cpp b/libminifi/src/processors/GetTCP.cpp
index 2b3b93c0..1d1d344a 100644
--- a/libminifi/src/processors/GetTCP.cpp
+++ b/libminifi/src/processors/GetTCP.cpp
@@ -50,16 +50,35 @@ namespace processors {
 
 const char *DataHandler::SOURCE_ENDPOINT_ATTRIBUTE = "source.endpoint";
 
-core::Property GetTCP::EndpointList("endpoint-list", "A comma delimited list of the endpoints to connect to. The format should be <server_address>:<port>.", "");
-core::Property GetTCP::ConcurrentHandlers("concurrent-handler-count", "Number of concurrent handlers for this session", "1");
-core::Property GetTCP::ReconnectInterval("reconnect-interval", "The number of seconds to wait before attempting to reconnect to the endpoint.", "5s");
-core::Property GetTCP::ReceiveBufferSize("receive-buffer-size", "The size of the buffer to receive data in. Default 16384 (16MB).", "16MB");
-core::Property GetTCP::SSLContextService("SSL Context Service", "SSL Context Service Name", "");
-core::Property GetTCP::StayConnected("Stay Connected", "Determines if we keep the same socket despite having no data", "true");
-core::Property GetTCP::ConnectionAttemptLimit("connection-attempt-timeout", "Maximum number of connection attempts before attempting backup hosts, if configured.", "3");
+core::Property GetTCP::EndpointList(
+    core::PropertyBuilder::createProperty("endpoint-list")->withDescription("A comma delimited list of the endpoints to connect to. The format should be <server_address>:<port>.")->isRequired(true)
+        ->build());
+
+core::Property GetTCP::ConcurrentHandlers(
+    core::PropertyBuilder::createProperty("concurrent-handler-count")->withDescription("Number of concurrent handlers for this session")->withDefaultValue<int>(1)->build());
+
+core::Property GetTCP::ReconnectInterval(
+    core::PropertyBuilder::createProperty("reconnect-interval")->withDescription("The number of seconds to wait before attempting to reconnect to the endpoint.")
+        ->withDefaultValue<core::TimePeriodValue>("5 s")->build());
+
+core::Property GetTCP::ReceiveBufferSize(
+    core::PropertyBuilder::createProperty("receive-buffer-size")->withDescription("The size of the buffer to receive data in. Default 16384 (16MB).")->withDefaultValue<core::DataSizeValue>("16 MB")
+        ->build());
+
+core::Property GetTCP::SSLContextService(
+    core::PropertyBuilder::createProperty("SSL Context Service")->withDescription("SSL Context Service Name")->asType<minifi::controllers::SSLContextService>()->build());
+
+core::Property GetTCP::StayConnected(
+    core::PropertyBuilder::createProperty("Stay Connected")->withDescription("Determines if we keep the same socket despite having no data")->withDefaultValue<bool>(true)->build());
+
+core::Property GetTCP::ConnectionAttemptLimit(
+    core::PropertyBuilder::createProperty("connection-attempt-timeout")->withDescription("Maximum number of connection attempts before attempting backup hosts, if configured")->withDefaultValue<int>(
+        3)->build());
+
 core::Property GetTCP::EndOfMessageByte(
-    "end-of-message-byte",
-    "Byte value which denotes end of message. Must be specified as integer within the valid byte range  (-128 thru 127). For example, '13' = Carriage return and '10' = New line. Default '13'.", "13");
+    core::PropertyBuilder::createProperty("end-of-message-byte")->withDescription(
+        "Byte value which denotes end of message. Must be specified as integer within the valid byte range  (-128 thru 127). For example, '13' = Carriage return and '10' = New line. Default '13'.")
+        ->withDefaultValue("13")->build());
 
 core::Relationship GetTCP::Success("success", "All files are routed to success");
 core::Relationship GetTCP::Partial("partial", "Indicates an incomplete message as a result of encountering the end of message byte trigger");
@@ -109,9 +128,8 @@ void GetTCP::onSchedule(const std::shared_ptr<core::ProcessContext> &context, co
     endpoints = utils::StringUtils::split(value, ",");
   }
 
-  if (context->getProperty(ConcurrentHandlers.getName(), value)) {
-    int64_t handlers = 0;
-    core::Property::StringToInt(value, handlers);
+  int handlers = 0;
+  if (context->getProperty(ConcurrentHandlers.getName(), handlers)) {
     concurrent_handlers_ = handlers;
   }
 
@@ -120,33 +138,22 @@ void GetTCP::onSchedule(const std::shared_ptr<core::ProcessContext> &context, co
   } else {
     stay_connected_ = true;
   }
-  if (context->getProperty(ConnectionAttemptLimit.getName(), value)) {
-    int64_t connects = 0;
-    core::Property::StringToInt(value, connects);
+  int connects = 0;
+  if (context->getProperty(ConnectionAttemptLimit.getName(), connects)) {
     connection_attempt_limit_ = connects;
   }
-  if (context->getProperty(ReceiveBufferSize.getName(), value)) {
-    int64_t size = 0;
-    core::Property::StringToInt(value, size);
-    receive_buffer_size_ = size;
-  }
+  context->getProperty(ReceiveBufferSize.getName(), receive_buffer_size_);
 
   if (context->getProperty(EndOfMessageByte.getName(), value)) {
+    logger_->log_trace("EOM is passed in as %s", value);
     int64_t byteValue = 0;
     core::Property::StringToInt(value, byteValue);
     endOfMessageByte = byteValue & 0xFF;
   }
 
-  if (context->getProperty(ReconnectInterval.getName(), value)) {
-    int64_t msec;
-    core::TimeUnit unit;
-    if (core::Property::StringToTime(value, msec, unit) && core::Property::ConvertTimeUnitToMS(msec, unit, msec)) {
-      reconnect_interval_ = msec;
-      logger_->log_debug("successfully applied reconnect interval of %ll", reconnect_interval_);
-    }
-  } else {
-    reconnect_interval_ = 5000;
-  }
+  logger_->log_trace("EOM is defined as %i", endOfMessageByte);
+
+  context->getProperty(ReconnectInterval.getName(), reconnect_interval_);
 
   handler_ = std::unique_ptr<DataHandler>(new DataHandler(sessionFactory));
 
diff --git a/libminifi/src/processors/HashContent.cpp b/libminifi/src/processors/HashContent.cpp
index 962e70cd..b8c2587a 100644
--- a/libminifi/src/processors/HashContent.cpp
+++ b/libminifi/src/processors/HashContent.cpp
@@ -77,6 +77,7 @@ void HashContent::onTrigger(core::ProcessContext *, core::ProcessSession *sessio
   std::shared_ptr<core::FlowFile> flowFile = session->get();
 
   if (!flowFile) {
+    logger_->log_trace("No flow file");
     return;
   }
 
@@ -84,6 +85,7 @@ void HashContent::onTrigger(core::ProcessContext *, core::ProcessSession *sessio
     session->transfer(flowFile, Failure);
   }
 
+  logger_->log_trace("attempting read");
   ReadCallback cb(flowFile, *this);
   session->read(flowFile, &cb);
   session->transfer(flowFile, Success);
@@ -91,6 +93,7 @@ void HashContent::onTrigger(core::ProcessContext *, core::ProcessSession *sessio
 
 int64_t HashContent::ReadCallback::process(std::shared_ptr<io::BaseStream> stream) {
   // This throws in case algo is not found, but that's fine
+  parent_.logger_->log_trace("Searching for %s", parent_.algoName_);
   auto algo = HashAlgos.at(parent_.algoName_);
 
   const auto& ret_val = algo(stream);
diff --git a/libminifi/src/processors/ListenSyslog.cpp b/libminifi/src/processors/ListenSyslog.cpp
index aab3c672..a3b332dd 100644
--- a/libminifi/src/processors/ListenSyslog.cpp
+++ b/libminifi/src/processors/ListenSyslog.cpp
@@ -28,6 +28,7 @@
 #include "utils/StringUtils.h"
 #include "core/ProcessContext.h"
 #include "core/ProcessSession.h"
+#include "core/TypedValues.h"
 
 namespace org {
 namespace apache {
@@ -35,16 +36,36 @@ namespace nifi {
 namespace minifi {
 namespace processors {
 #ifndef WIN32
-core::Property ListenSyslog::RecvBufSize("Receive Buffer Size", "The size of each buffer used to receive Syslog messages.", "65507 B");
-core::Property ListenSyslog::MaxSocketBufSize("Max Size of Socket Buffer", "The maximum size of the socket buffer that should be used.", "1 MB");
-core::Property ListenSyslog::MaxConnections("Max Number of TCP Connections", "The maximum number of concurrent connections to accept Syslog messages in TCP mode.", "2");
-core::Property ListenSyslog::MaxBatchSize("Max Batch Size", "The maximum number of Syslog events to add to a single FlowFile.", "1");
-core::Property ListenSyslog::MessageDelimiter("Message Delimiter", "Specifies the delimiter to place between Syslog messages when multiple "
-                                              "messages are bundled together (see <Max Batch Size> core::Property).",
-                                              "\n");
-core::Property ListenSyslog::ParseMessages("Parse Messages", "Indicates if the processor should parse the Syslog messages. If set to false, each outgoing FlowFile will only.", "false");
-core::Property ListenSyslog::Protocol("Protocol", "The protocol for Syslog communication.", "UDP");
-core::Property ListenSyslog::Port("Port", "The port for Syslog communication.", "514");
+core::Property ListenSyslog::RecvBufSize(
+    core::PropertyBuilder::createProperty("Receive Buffer Size")->withDescription("The size of each buffer used to receive Syslog messages.")->
+    withDefaultValue<core::DataSizeValue>("65507 B")->build());
+
+core::Property ListenSyslog::MaxSocketBufSize(
+    core::PropertyBuilder::createProperty("Max Size of Socket Buffer")->withDescription("The maximum size of the socket buffer that should be used.")->withDefaultValue<core::DataSizeValue>("1 MB")
+        ->build());
+
+core::Property ListenSyslog::MaxConnections(
+    core::PropertyBuilder::createProperty("Max Number of TCP Connections")->withDescription("The maximum number of concurrent connections to accept Syslog messages in TCP mode.")
+        ->withDefaultValue<int>(2)->build());
+
+core::Property ListenSyslog::MaxBatchSize(
+    core::PropertyBuilder::createProperty("Max Batch Size")->withDescription("The maximum number of Syslog events to add to a single FlowFile.")->withDefaultValue<int>(1)->build());
+
+core::Property ListenSyslog::MessageDelimiter(
+    core::PropertyBuilder::createProperty("Message Delimiter")->withDescription("Specifies the delimiter to place between Syslog messages when multiple "
+                                                                                "messages are bundled together (see <Max Batch Size> core::Property).")->withDefaultValue("\n")->build());
+
+core::Property ListenSyslog::ParseMessages(
+    core::PropertyBuilder::createProperty("Parse Messages")->withDescription("Indicates if the processor should parse the Syslog messages. If set to false, each outgoing FlowFile will only.")
+        ->withDefaultValue<bool>(false)->build());
+
+core::Property ListenSyslog::Protocol(
+    core::PropertyBuilder::createProperty("Protocol")->withDescription("The protocol for Syslog communication.")->withAllowableValue<std::string>("UDP")->withAllowableValue("TCP")->withDefaultValue(
+        "UDP")->build());
+
+core::Property ListenSyslog::Port(
+    core::PropertyBuilder::createProperty("Port")->withDescription("The port for Syslog communication")->withDefaultValue<int64_t>(514, core::StandardValidators::PORT_VALIDATOR())->build());
+
 core::Relationship ListenSyslog::Success("success", "All files are routed to success");
 core::Relationship ListenSyslog::Invalid("invalid", "SysLog message format invalid");
 
diff --git a/libminifi/src/processors/LogAttribute.cpp b/libminifi/src/processors/LogAttribute.cpp
index cec91911..ee94ece7 100644
--- a/libminifi/src/processors/LogAttribute.cpp
+++ b/libminifi/src/processors/LogAttribute.cpp
@@ -38,13 +38,22 @@ namespace apache {
 namespace nifi {
 namespace minifi {
 namespace processors {
-core::Property LogAttribute::LogLevel("Log Level", "The Log Level to use when logging the Attributes", "info");
-core::Property LogAttribute::AttributesToLog("Attributes to Log", "A comma-separated list of Attributes to Log. If not specified, all attributes will be logged.", "");
-core::Property LogAttribute::AttributesToIgnore("Attributes to Ignore", "A comma-separated list of Attributes to ignore. If not specified, no attributes will be ignored.", "");
-core::Property LogAttribute::LogPayload("Log Payload", "If true, the FlowFile's payload will be logged, in addition to its attributes;"
-                                        "otherwise, just the Attributes will be logged.",
-                                        "false");
-core::Property LogAttribute::LogPrefix("Log prefix", "Log prefix appended to the log lines. It helps to distinguish the output of multiple LogAttribute processors.", "");
+
+core::Property LogAttribute::LogLevel(core::PropertyBuilder::createProperty("Log Level")->withDescription("The Log Level to use when logging the Attributes")->withAllowableValues<std::string>({
+    "info", "trace", "error", "warn", "debug" })->build());
+
+core::Property LogAttribute::AttributesToLog(
+    core::PropertyBuilder::createProperty("Attributes to Log")->withDescription("A comma-separated list of Attributes to Log. If not specified, all attributes will be logged.")->build());
+
+core::Property LogAttribute::AttributesToIgnore(
+    core::PropertyBuilder::createProperty("Attributes to Ignore")->withDescription("A comma-separated list of Attributes to ignore. If not specified, no attributes will be ignored.")->build());
+
+core::Property LogAttribute::LogPayload(core::PropertyBuilder::createProperty("Log Payload")->withDescription("If true, the FlowFile's payload will be logged, in addition to its attributes."
+                                                                                                              "otherwise, just the Attributes will be logged")->withDefaultValue<bool>(false)->build());
+
+core::Property LogAttribute::LogPrefix(
+    core::PropertyBuilder::createProperty("Log Prefix")->withDescription("Log prefix appended to the log lines. It helps to distinguish the output of multiple LogAttribute processors.")->build());
+
 core::Relationship LogAttribute::Success("success", "success operational on the flow record");
 
 void LogAttribute::initialize() {
@@ -82,9 +91,8 @@ void LogAttribute::onTrigger(core::ProcessContext *context, core::ProcessSession
   if (context->getProperty(LogPrefix.getName(), value)) {
     dashLine = "-----" + value + "-----";
   }
-  if (context->getProperty(LogPayload.getName(), value)) {
-    org::apache::nifi::minifi::utils::StringUtils::StringToBool(value, logPayload);
-  }
+
+  context->getProperty(LogPayload.getName(), logPayload);
 
   message << "Logging for flow file " << "\n";
   message << dashLine;
diff --git a/libminifi/src/processors/PutFile.cpp b/libminifi/src/processors/PutFile.cpp
index fa461d4a..b2790107 100644
--- a/libminifi/src/processors/PutFile.cpp
+++ b/libminifi/src/processors/PutFile.cpp
@@ -46,12 +46,18 @@ std::shared_ptr<utils::IdGenerator> PutFile::id_generator_ = utils::IdGenerator:
 
 core::Property PutFile::Directory(
     core::PropertyBuilder::createProperty("Directory")->withDescription("The output directory to which to put files")->supportsExpressionLanguage(true)->withDefaultValue(".")->build());
-core::Property PutFile::ConflictResolution("Conflict Resolution Strategy", "Indicates what should happen when a file with the same name already exists in the output directory",
-                                           CONFLICT_RESOLUTION_STRATEGY_FAIL);
+
+core::Property PutFile::ConflictResolution(
+    core::PropertyBuilder::createProperty("Conflict Resolution Strategy")->withDescription("Indicates what should happen when a file with the same name already exists in the output directory")
+        ->withAllowableValue<std::string>(CONFLICT_RESOLUTION_STRATEGY_FAIL)->withAllowableValue(CONFLICT_RESOLUTION_STRATEGY_IGNORE)->withAllowableValue(CONFLICT_RESOLUTION_STRATEGY_REPLACE)
+        ->withDefaultValue(CONFLICT_RESOLUTION_STRATEGY_FAIL)->build());
+
 core::Property PutFile::CreateDirs("Create Missing Directories", "If true, then missing destination directories will be created. "
                                    "If false, flowfiles are penalized and sent to failure.",
                                    "true", true, "", { "Directory" }, { });
-core::Property PutFile::MaxDestFiles("Maximum File Count", "Specifies the maximum number of files that can exist in the output directory", "-1");
+
+core::Property PutFile::MaxDestFiles(
+    core::PropertyBuilder::createProperty("Maximum File Count")->withDescription("Specifies the maximum number of files that can exist in the output directory")->withDefaultValue<int>(-1)->build());
 
 core::Relationship PutFile::Success("success", "All files are routed to success");
 core::Relationship PutFile::Failure("failure", "Failed files (conflict, write failure, etc.) are transferred to failure");
diff --git a/libminifi/src/processors/RouteOnAttribute.cpp b/libminifi/src/processors/RouteOnAttribute.cpp
index 3e2bf61b..ddef2320 100644
--- a/libminifi/src/processors/RouteOnAttribute.cpp
+++ b/libminifi/src/processors/RouteOnAttribute.cpp
@@ -48,7 +48,7 @@ void RouteOnAttribute::onDynamicPropertyModified(const core::Property &orig_prop
     core::Relationship route_rel { route.first, "Dynamic route" };
     route_rels_[route.first] = route_rel;
     relationships.insert(route_rel);
-    logger_->log_info("RouteOnAttribute registered route '%s' with expression '%s'", route.first, route.second.getValue());
+    logger_->log_info("RouteOnAttribute registered route '%s' with expression '%s'", route.first, route.second.getValue().to_string());
   }
 
   relationships.insert(Unmatched);
diff --git a/libminifi/test/unit/ControllerTests.cpp b/libminifi/test/unit/ControllerTests.cpp
index 0e755664..f3df5ce9 100644
--- a/libminifi/test/unit/ControllerTests.cpp
+++ b/libminifi/test/unit/ControllerTests.cpp
@@ -36,9 +36,14 @@ class TestStateController : public minifi::state::StateController {
   virtual ~TestStateController() {
   }
 
-  virtual std::string getComponentName() {
+  virtual std::string getComponentName() const {
     return "TestStateController";
   }
+
+  virtual std::string getComponentUUID() const {
+    return "uuid";
+  }
+
   /**
    * Start the client
    */
@@ -85,9 +90,13 @@ class TestUpdateSink : public minifi::state::StateMonitor {
     return vec;
   }
 
-  virtual std::string getComponentName() {
+  virtual std::string getComponentName() const {
     return "TestUpdateSink";
   }
+
+  virtual std::string getComponentUUID() const {
+      return "uuid";
+    }
   /**
    * Start the client
    */
diff --git a/libminifi/test/unit/GetTCPTests.cpp b/libminifi/test/unit/GetTCPTests.cpp
index 777ffd5f..270a170d 100644
--- a/libminifi/test/unit/GetTCPTests.cpp
+++ b/libminifi/test/unit/GetTCPTests.cpp
@@ -170,7 +170,8 @@ TEST_CASE("GetTCPWithOEM", "[GetTCP2]") {
 
   LogTestController::getInstance().setDebug<minifi::processors::LogAttribute>();
   LogTestController::getInstance().setTrace<core::repository::VolatileContentRepository >();
-  LogTestController::getInstance().setDebug<minifi::processors::GetTCP>();
+  LogTestController::getInstance().setTrace<minifi::processors::GetTCP>();
+  LogTestController::getInstance().setTrace<core::ConfigurableComponent>();
   LogTestController::getInstance().setTrace<minifi::io::Socket>();
 
   std::shared_ptr<core::Repository> repo = std::make_shared<TestRepository>();
diff --git a/libminifi/test/unit/ManifestTests.cpp b/libminifi/test/unit/ManifestTests.cpp
index 7e463dd7..130966fb 100644
--- a/libminifi/test/unit/ManifestTests.cpp
+++ b/libminifi/test/unit/ManifestTests.cpp
@@ -39,7 +39,7 @@ TEST_CASE("Test Required", "[required]") {
   REQUIRE(prop_descriptors.children.size() > 0);
   const auto &prop_0 = prop_descriptors.children[0];
   REQUIRE(prop_0.children.size() >= 3);
-  const auto &prop_0_required = prop_0.children[2];
+  const auto &prop_0_required = prop_0.children[3];
   REQUIRE("required" == prop_0_required.name);
   REQUIRE(!std::dynamic_pointer_cast<minifi::state::response::BoolValue>(prop_0_required.value.getValue())->getValue());
 }
@@ -59,11 +59,13 @@ TEST_CASE("Test Valid Regex", "[validRegex]") {
   const auto &prop_0 = prop_descriptors.children[0];
   REQUIRE(prop_0.children.size() >= 3);
   const auto &df = prop_0.children[3];
-  REQUIRE("expressionLanguageScope" == df.name);
-  const auto &prop_0_defv = prop_0.children[4];
-  REQUIRE("defaultValue" == prop_0_defv.name);
+  REQUIRE("required" == df.name);
+  const auto &prop_0_els = prop_0.children[4];
+  REQUIRE("expressionLanguageScope" == prop_0_els.name);
   const auto &prop_0_valid_regex = prop_0.children[5];
-  REQUIRE("validRegex" == prop_0_valid_regex.name);
+  REQUIRE("defaultValue" == prop_0_valid_regex.name);
+  const auto &prop_0_defv = prop_0.children[6];
+  REQUIRE("validRegex" == prop_0_defv.name);
 }
 
 TEST_CASE("Test Relationships", "[rel1]") {
@@ -112,8 +114,9 @@ TEST_CASE("Test Dependent", "[dependent]") {
   REQUIRE(prop_descriptors.children.size() > 0);
   const auto &prop_0 = prop_descriptors.children[1];
   REQUIRE(prop_0.children.size() >= 3);
-  REQUIRE("expressionLanguageScope" == prop_0.children[3].name);
-  REQUIRE("defaultValue" == prop_0.children[4].name);
+  REQUIRE("required" == prop_0.children[3].name);
+  REQUIRE("expressionLanguageScope" == prop_0.children[4].name);
+  REQUIRE("defaultValue" == prop_0.children[5].name);
   const auto &prop_0_dependent_0 = prop_descriptors.children[2];
   REQUIRE("Directory" == prop_0_dependent_0.name);
 }
diff --git a/libminifi/test/unit/NetworkPrioritizerServiceTests.cpp b/libminifi/test/unit/NetworkPrioritizerServiceTests.cpp
index 765487e4..54babe79 100644
--- a/libminifi/test/unit/NetworkPrioritizerServiceTests.cpp
+++ b/libminifi/test/unit/NetworkPrioritizerServiceTests.cpp
@@ -35,8 +35,8 @@ TEST_CASE("TestPrioritizerOneInterface", "[test1]") {
   controller->initialize();
   controller->setProperty(minifi::controllers::NetworkPrioritizerService::NetworkControllers, "eth0,eth1");
   controller->setProperty(minifi::controllers::NetworkPrioritizerService::VerifyInterfaces, "false");
-  controller->setProperty(minifi::controllers::NetworkPrioritizerService::MaxThroughput, "10");
-  controller->setProperty(minifi::controllers::NetworkPrioritizerService::MaxPayload, "10");
+  controller->setProperty(minifi::controllers::NetworkPrioritizerService::MaxThroughput, "10 B");
+  controller->setProperty(minifi::controllers::NetworkPrioritizerService::MaxPayload, "10 B");
   controller->onEnable();
   REQUIRE("eth0" == controller->getInterface(0).getInterface());
 }
@@ -47,8 +47,8 @@ TEST_CASE("TestPrioritizerOneInterfaceMaxPayload", "[test2]") {
   controller->initialize();
   controller->setProperty(minifi::controllers::NetworkPrioritizerService::NetworkControllers, "eth0,eth1");
   controller->setProperty(minifi::controllers::NetworkPrioritizerService::VerifyInterfaces, "false");
-  controller->setProperty(minifi::controllers::NetworkPrioritizerService::MaxThroughput, "1");
-  controller->setProperty(minifi::controllers::NetworkPrioritizerService::MaxPayload, "1");
+  controller->setProperty(minifi::controllers::NetworkPrioritizerService::MaxThroughput, "1 B");
+  controller->setProperty(minifi::controllers::NetworkPrioritizerService::MaxPayload, "1 B");
   controller->onEnable();
   // can't because we've triggered the max payload
   REQUIRE("" == controller->getInterface(5).getInterface());
@@ -60,7 +60,7 @@ TEST_CASE("TestPrioritizerOneInterfaceMaxThroughput", "[test3]") {
   controller->initialize();
   controller->setProperty(minifi::controllers::NetworkPrioritizerService::NetworkControllers, "eth0,eth1");
   controller->setProperty(minifi::controllers::NetworkPrioritizerService::VerifyInterfaces, "false");
-  controller->setProperty(minifi::controllers::NetworkPrioritizerService::MaxThroughput, "10");
+  controller->setProperty(minifi::controllers::NetworkPrioritizerService::MaxThroughput, "10 B");
   controller->onEnable();
   // can't because we've triggered the max payload
   REQUIRE("eth0" == controller->getInterface(5).getInterface());
@@ -71,6 +71,8 @@ TEST_CASE("TestPrioritizerOneInterfaceMaxThroughput", "[test3]") {
 }
 
 TEST_CASE("TestPriorotizerMultipleInterfaces", "[test4]") {
+  LogTestController::getInstance().setTrace<minifi::controllers::NetworkPrioritizerService>();
+
   auto controller = std::make_shared<minifi::controllers::NetworkPrioritizerService>("TestService");
   auto controller2 = std::make_shared<minifi::controllers::NetworkPrioritizerService>("TestService2");
   auto controller3 = std::make_shared<minifi::controllers::NetworkPrioritizerService>("TestService3");
@@ -81,13 +83,13 @@ TEST_CASE("TestPriorotizerMultipleInterfaces", "[test4]") {
   controller3->initialize();
   controller3->setProperty(minifi::controllers::NetworkPrioritizerService::NetworkControllers, "eth0");
   controller3->setProperty(minifi::controllers::NetworkPrioritizerService::VerifyInterfaces, "false");
-  controller3->setProperty(minifi::controllers::NetworkPrioritizerService::MaxThroughput, "10");
+  controller3->setProperty(minifi::controllers::NetworkPrioritizerService::MaxThroughput, "10 B");
   controller3->onEnable();
 
   controller2->initialize();
   controller2->setProperty(minifi::controllers::NetworkPrioritizerService::NetworkControllers, "eth1");
   controller2->setProperty(minifi::controllers::NetworkPrioritizerService::VerifyInterfaces, "false");
-  controller2->setProperty(minifi::controllers::NetworkPrioritizerService::MaxThroughput, "10");
+  controller2->setProperty(minifi::controllers::NetworkPrioritizerService::MaxThroughput, "10 B");
   controller2->onEnable();
   std::vector<std::shared_ptr<core::controller::ControllerService> > services;
   services.push_back(controller2);
@@ -112,13 +114,13 @@ TEST_CASE("TestPriorotizerMultipleInterfacesNeverSwitch", "[test5]") {
   controller3->initialize();
   controller3->setProperty(minifi::controllers::NetworkPrioritizerService::NetworkControllers, "eth0");
   controller3->setProperty(minifi::controllers::NetworkPrioritizerService::VerifyInterfaces, "false");
-  controller3->setProperty(minifi::controllers::NetworkPrioritizerService::MaxThroughput, "1000");
+  controller3->setProperty(minifi::controllers::NetworkPrioritizerService::MaxThroughput, "1 kB");
   controller3->onEnable();
 
   controller2->initialize();
   controller2->setProperty(minifi::controllers::NetworkPrioritizerService::NetworkControllers, "eth1");
   controller2->setProperty(minifi::controllers::NetworkPrioritizerService::VerifyInterfaces, "false");
-  controller2->setProperty(minifi::controllers::NetworkPrioritizerService::MaxThroughput, "10");
+  controller2->setProperty(minifi::controllers::NetworkPrioritizerService::MaxThroughput, "10 B");
   controller2->onEnable();
   std::vector<std::shared_ptr<core::controller::ControllerService> > services;
   services.push_back(controller3);
@@ -147,15 +149,15 @@ TEST_CASE("TestPriorotizerMultipleInterfacesMaxPayload", "[test4]") {
   controller3->initialize();
   controller3->setProperty(minifi::controllers::NetworkPrioritizerService::NetworkControllers, "eth0");
   controller3->setProperty(minifi::controllers::NetworkPrioritizerService::VerifyInterfaces, "false");
-  controller3->setProperty(minifi::controllers::NetworkPrioritizerService::MaxThroughput, "1000");
+  controller3->setProperty(minifi::controllers::NetworkPrioritizerService::MaxThroughput, "1 kB");
 
   controller3->onEnable();
 
   controller2->initialize();
   controller2->setProperty(minifi::controllers::NetworkPrioritizerService::NetworkControllers, "eth1");
   controller2->setProperty(minifi::controllers::NetworkPrioritizerService::VerifyInterfaces, "false");
-  controller2->setProperty(minifi::controllers::NetworkPrioritizerService::MaxThroughput, "10");
-  controller3->setProperty(minifi::controllers::NetworkPrioritizerService::MaxPayload, "10");
+  controller2->setProperty(minifi::controllers::NetworkPrioritizerService::MaxThroughput, "10 B");
+  controller3->setProperty(minifi::controllers::NetworkPrioritizerService::MaxPayload, "10 B");
   controller2->onEnable();
   std::vector<std::shared_ptr<core::controller::ControllerService> > services;
   services.push_back(controller2);
diff --git a/libminifi/test/unit/YamlConfigurationTests.cpp b/libminifi/test/unit/YamlConfigurationTests.cpp
index 71ef512c..12ce4236 100644
--- a/libminifi/test/unit/YamlConfigurationTests.cpp
+++ b/libminifi/test/unit/YamlConfigurationTests.cpp
@@ -471,7 +471,7 @@ TEST_CASE("Test Dynamic Unsupported", "[YamlConfigurationDynamicUnsupported]") {
 
   LogTestController &logTestController = LogTestController::getInstance();
   logTestController.setDebug<TestPlan>();
-  logTestController.setDebug<core::YamlConfiguration>();
+  logTestController.setTrace<core::YamlConfiguration>();
 
   std::shared_ptr<core::Repository> testProvRepo = core::createRepository("provenancerepository", true);
   std::shared_ptr<core::Repository> testFlowFileRepo = core::createRepository("flowfilerepository", true);
@@ -487,7 +487,7 @@ Flow Controller:
 - name: PutFile
   class: PutFile
   Properties:
-    Dynamic Property: Bad
+     Dynamic Property: Bad
       )";
   std::istringstream configYamlStream(TEST_CONFIG_YAML);
   std::unique_ptr<core::ProcessGroup> rootFlowConfig = yamlConfig.getYamlRoot(configYamlStream);
diff --git a/nanofi/examples/CMakeLists.txt b/nanofi/examples/CMakeLists.txt
index 16410617..7cad4e4f 100644
--- a/nanofi/examples/CMakeLists.txt
+++ b/nanofi/examples/CMakeLists.txt
@@ -57,6 +57,8 @@ else ()
     set(LINK_END_FLAGS "")
 endif ()
 
+if (NOT WIN32)
+
 add_executable(generate_flow generate_flow.c)
 
 add_executable(terminate_handler terminate_handler.c)
@@ -65,8 +67,6 @@ target_link_libraries(generate_flow nanofi ${CMAKE_THREAD_LIBS_INIT} ${LINK_FLAG
 
 target_link_libraries(terminate_handler nanofi ${CMAKE_THREAD_LIBS_INIT} )
 
-if (NOT WIN32)
-
 add_executable(transmit_flow transmit_flow.c)
 
 target_link_libraries(transmit_flow nanofi ${CMAKE_THREAD_LIBS_INIT}  ${LINK_FLAGS} minifi-http-curl ${LINK_END_FLAGS})
diff --git a/nanofi/include/cxx/C2CallbackAgent.h b/nanofi/include/cxx/C2CallbackAgent.h
index f620baa2..c48f7791 100644
--- a/nanofi/include/cxx/C2CallbackAgent.h
+++ b/nanofi/include/cxx/C2CallbackAgent.h
@@ -25,8 +25,8 @@
 #include <mutex>
 #include <thread>
 
-#include "c2/C2Agent.h"
 #include "core/state/Value.h"
+#include "c2/C2Agent.h"
 #include "c2/C2Payload.h"
 #include "c2/C2Protocol.h"
 #include "io/validation.h"


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services