You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by fg...@apache.org on 2022/03/08 18:59:56 UTC

[nifi-minifi-cpp] branch main updated: MINIFICPP-1736 Add MiNiFi agent C2 operation capabilities to agent manifest

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

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


The following commit(s) were added to refs/heads/main by this push:
     new ea6222a  MINIFICPP-1736 Add MiNiFi agent C2 operation capabilities to agent manifest
ea6222a is described below

commit ea6222aa79a1f12e4486d2a2cb9d91c3f02f59a1
Author: Gabor Gyimesi <ga...@gmail.com>
AuthorDate: Fri Jan 21 16:36:47 2022 +0100

    MINIFICPP-1736 Add MiNiFi agent C2 operation capabilities to agent manifest
    
    - Add describe operation to agent manifest
    - Add Update operands to supported opretions
    - Add transfer operation to agent manifest
    - Remove unused VALIDATE operation
    - Add clear operands to agent manifest
    - Add operands for START and STOP operation
    - Centralize configuration options in Configuration class
    - Add metadata for update operation
    - Filter agent manifest update properties by updatepolicycontroller
    
    Signed-off-by: Ferenc Gerlits <fg...@gmail.com>
    This closes #1253
---
 C2.md                                              |  86 ++---
 controller/Controller.h                            |  11 +-
 encrypt-config/ConfigFile.cpp                      |  11 +-
 encrypt-config/tests/ConfigFileEncryptorTests.cpp  |  26 +-
 encrypt-config/tests/ConfigFileTests.cpp           |  29 +-
 extensions/civetweb/protocols/RESTReceiver.cpp     |   5 +-
 .../coap/controllerservice/CoapConnector.cpp       |   2 +-
 extensions/coap/protocols/CoapC2Protocol.cpp       |   2 +-
 extensions/coap/tests/CoapC2VerifyHeartbeat.cpp    |  16 +-
 extensions/coap/tests/CoapIntegrationBase.h        |   2 +-
 extensions/http-curl/protocols/RESTSender.cpp      |  11 +-
 extensions/http-curl/tests/AbsoluteTimeoutTest.cpp |   1 +
 extensions/http-curl/tests/C2DebugBundleTest.cpp   |   3 +-
 .../http-curl/tests/C2DescribeManifestTest.cpp     |   2 +-
 extensions/http-curl/tests/C2LogHeartbeatTest.cpp  |   3 +-
 extensions/http-curl/tests/C2NullConfiguration.cpp |  39 ++-
 extensions/http-curl/tests/C2PauseResumeTest.cpp   |   2 +-
 extensions/http-curl/tests/C2RequestClassTest.cpp  |   9 +-
 .../http-curl/tests/C2VerifyHeartbeatAndStop.cpp   |   5 +-
 .../C2VerifyResourceConsumptionInHeartbeat.cpp     |   3 +-
 .../http-curl/tests/C2VerifyServeResults.cpp       |  15 +-
 extensions/http-curl/tests/HTTPHandlers.h          | 145 +++++++-
 extensions/http-curl/tests/HTTPIntegrationBase.h   |  41 +--
 extensions/http-curl/tests/HTTPSiteToSiteTests.cpp |   7 +-
 .../http-curl/tests/HttpPostIntegrationTest.cpp    |   5 +-
 .../http-curl/tests/TimeoutHTTPSiteToSiteTests.cpp |   7 +-
 extensions/http-curl/tests/VerifyInvokeHTTP.h      |   2 +-
 extensions/jni/JVMCreator.h                        |   7 +-
 extensions/mqtt/protocol/MQTTC2Protocol.cpp        |   7 +-
 extensions/script/python/PythonCreator.h           |   3 +-
 .../tests/integration/SecureSocketGetTCPTest.cpp   |  12 +-
 libminifi/include/c2/C2Payload.h                   |  25 +-
 libminifi/include/core/ProcessContext.h            |   8 +-
 libminifi/include/core/Property.h                  |  11 +
 .../core/controller/ControllerServiceLookup.h      |   4 +-
 .../core/controller/ControllerServiceProvider.h    |  12 +-
 .../ForwardingControllerServiceProvider.h          |   6 +-
 .../include/core/extension/ExtensionManager.h      |   6 -
 libminifi/include/core/state/Value.h               |   1 +
 .../include/core/state/nodes/AgentInformation.h    |  38 +-
 .../include/core/state/nodes/SupportedOperations.h |  70 ++++
 libminifi/include/properties/Configuration.h       |  72 +++-
 libminifi/src/Configuration.cpp                    | 175 ++++++----
 libminifi/src/Configure.cpp                        |   5 +-
 libminifi/src/FlowController.cpp                   |   1 +
 libminifi/src/c2/C2Agent.cpp                       | 381 ++++++++++++---------
 libminifi/src/c2/C2Client.cpp                      |  10 +-
 libminifi/src/c2/HeartbeatJsonSerializer.cpp       |   7 +-
 libminifi/src/c2/protocols/RESTProtocol.cpp        |   5 +-
 .../core/controller/ControllerServiceProvider.cpp  |   2 +-
 libminifi/src/core/extension/ExtensionManager.cpp  |   7 +-
 .../src/core/state/nodes/SupportedOperations.cpp   | 146 ++++++++
 libminifi/src/utils/ChecksumCalculator.cpp         |   3 +-
 libminifi/test/TestBase.h                          |   5 +-
 libminifi/test/resources/TestHTTPGet.yml           |  10 +-
 libminifi/test/sensors-tests/SensorTests.cpp       |   2 +-
 libminifi/test/unit/DecryptorTests.cpp             |   2 +-
 libminifi/test/unit/SocketTests.cpp                |   5 +-
 nanofi/include/cxx/Instance.h                      |   4 +-
 59 files changed, 1067 insertions(+), 475 deletions(-)

diff --git a/C2.md b/C2.md
index f55c1c3..37be50a 100644
--- a/C2.md
+++ b/C2.md
@@ -17,7 +17,7 @@
 
 
 This readme defines the Command and control configuration options that work with Apache NiFi. All
-options defined are located in minifi.properties. 
+options defined are located in minifi.properties.
 
 ## Table of Contents
 
@@ -33,10 +33,10 @@ options defined are located in minifi.properties.
 ## Description
 
 Apache NiFi MiNiFi C++ can communicates with a C2 Server via a number of protocols. These protocols
-deliver a C2 response the server, expecting requests in a heartbeat response. The protocol transforms 
-the C2 messages into a protocol specific representation. The internal representation is an AST therefore 
-you must define the root classes, which configure the classes that branch from the root. You can define 
-arbitrary nodes and sub-trees, but this isn't necessary and only advantageous for custom C2 servers. That 
+deliver a C2 response the server, expecting requests in a heartbeat response. The protocol transforms
+the C2 messages into a protocol specific representation. The internal representation is an AST therefore
+you must define the root classes, which configure the classes that branch from the root. You can define
+arbitrary nodes and sub-trees, but this isn't necessary and only advantageous for custom C2 servers. That
 will be explained in greater detail in the metrics section.
 
 ## Configuration
@@ -67,30 +67,30 @@ be requested via C2 DESCRIBE manifest command.
 	#   ConfigurationChecksums: hashes of the configuration files; can be used to detect unexpected modifications
 	# the default is
 	nifi.c2.root.classes=DeviceInfoNode,AgentInformation,FlowInformation
-	
+
 	# specify C2 protocol -- default is RESTSender if not specified
 	nifi.c2.agent.protocol.class=RESTSender
 	# may also use MQTT or CoapProtocol
 	# nifi.c2.agent.protocol.class=MQTTC2Protocol
 	# nifi.c2.agent.protocol.class=CoapProtocol
-	
-	# control c2 heartbeat interval in millisecocnds
+
+	# control c2 heartbeat interval in milliseconds
 	nifi.c2.agent.heartbeat.period=250
-	
+
 	# enable reporter classes
 	nifi.c2.agent.heartbeat.reporter.class=RESTReciver
-	
+
 	# specify the rest URIs if using RESTSender
 	nifi.c2.rest.url=http://<your-c2-server>/<c2-api-path>/c2-protocol/heartbeat
 	nifi.c2.rest.url.ack=http://<your-c2-server>/<c2-api-path>/c2-protocol/acknowledge
 	nifi.c2.flow.base.url=http://<your-c2-server>/<c2-api-path>/c2-protocol/
-	
+
 	# c2 agent identifier -- must be defined to run agent
 	nifi.c2.agent.identifier=<your identifier>
-	
+
 	# c2 agent class -- must be defined to run agent
 	nifi.c2.agent.class=<your agent class>
-	
+
 	# configure SSL Context service for REST Protocol
 	#nifi.c2.rest.ssl.context.service
 
@@ -103,8 +103,8 @@ are defined apriori and may reference a metrics class specific to a processor. T
 a configuration of an agent
 
 	# in minifi.properties
- 	
-	
+
+
 	nifi.c2.root.class.definitions=metrics
 	nifi.c2.root.class.definitions.metrics.name=metrics
 	nifi.c2.root.class.definitions.metrics.metrics=typedmetrics,processorMetrics
@@ -112,13 +112,13 @@ a configuration of an agent
 	nifi.c2.root.class.definitions.metrics.metrics.typedmetrics.classes=ProcessMetrics,SystemInformation
 	nifi.c2.root.class.definitions.metrics.metrics.processorMetrics.name=ProcessorMetrics
 	nifi.c2.root.class.definitions.metrics.metrics.processorMetrics.classes=GetFileMetrics
-	
+
 This example shows a metrics sub tree defined by the option 'nifi.c2.root.class.definitions'.
 
 This is a comma separated list of all sub trees. In the example, above, only one sub tree exists: metrics.
 
 The options below metrics define the sub-trees within metrics: typedmetrics and processorMetrics. Each of these has a name.
-The classes sub option will define the metrics classes that are placed within this sub-tree. For the RESTProtocol, the above 
+The classes sub option will define the metrics classes that are placed within this sub-tree. For the RESTProtocol, the above
 configuration produces the following JSON:
 
 	"metrics": {
@@ -148,22 +148,22 @@ configuration produces the following JSON:
             }
         }
     }
-    
+
 
 ### Protocols
 
-The default protocol is a RESTFul service; however, there is an MQTT protocol with a translation to use the 
+The default protocol is a RESTFul service; however, there is an MQTT protocol with a translation to use the
 RESTFul C2 server and a CoAP Protocol implementation. The CoAP protocol requires that COAP be enabled either
 through the bootstrap or the cmake flag -DENABLE_COAP=ON .
 
 Once configured, COAP uses a controller service within the flow OR minifi properties entries: nifi.c2.agent.coap.host and nifi.c2.agent.coap.port.
 Note that with CoAP, the payload will be significantly smaller, paring down metrics that are sent in each heartbeat. This will be useful for
-constrained environments. 
+constrained environments.
 
 	nifi.c2.agent.coap.host=hostname
 	nifi.c2.agent.coap.port=<port number>
-   
-   
+
+
 If you wish to use the Controller service you made add a controller service named CoapConnectorService with the properties in the example config
 below. Note that Max Queue Size is the only non-required property:
 
@@ -181,7 +181,7 @@ you may use the ConvertBase, ConvertHeartBeat, ConvertJSONAack, and ConvertUpdat
 State is not kept with an intermediate agent other than the broker. The broker is not embedded with the agent to simplify the agent.
 
 An example configuration, below, defines an agent that receives and forward MQTT C2 requests to a C2 server. Additionally, this agent
-will forward responses and updates to the heartbeating agents. 
+will forward responses and updates to the heartbeating agents.
 
 	MiNiFi Config Version: 3
 	Flow Controller:
@@ -329,38 +329,38 @@ will forward responses and updates to the heartbeating agents.
 ### UpdatePolicies
 
 Updates to MiNiFi C++ properties can be controlled through an UpdatePolicyControllerService named
-C2UpdatePolicy. The service supports several configuration options. They are defined in the following example:   
+C2UpdatePolicy. The service supports several configuration options. They are defined in the following example:
+
 
-	
 	Controller Services:
 	- id: 94491a38-015a-1000-0000-000000000001
 	  name: C2UpdatePolicy
 	  class: UpdatePolicyControllerService
 	  Properties:
-	    # true enables all properties to be updated. 
+	    # true enables all properties to be updated.
 	    Allow All Properties: true
 	    # allowed properties are those which can be updated
-	    Allowed Properties: 
-	             Property_1:true
-	             Property_2:true
+	    Allowed Properties:
+	      - value: Property_1
+	      - value: Property_2
 	    Disallowed Properties:
-	    		 Property_3:true
-	             Property_4:true
-	
- 	
+	      - value: Property_3
+	      - value: Property_4
+
+
 ### Triggers
-  
+
   C2 Triggers can be activated to perform some C2 activity via a local event. Currently only FileUpdateTrigger exists, which monitors
   for C2 File triggers to update the flow configuration. Classes can be defined as a comma separated list of classes to load via the option
   nifi.c2.agent.trigger.classes
-  
-  
+
+
 #### C2 File triggers
 
 C2 updates can be triggered with updates to a flow configuration file. It doesn't have to be the same base configuration file. It
 will be copied into place. A new property, nifi.c2.file.watch, can be placed into minifi.properties to monitor. If the update time
 changes while the agent is running, it will be copied into place of the file defined by nifi.flow.configuration.file. The agent
-will then be restarted with the new flow configuration. If a failure occurs in reading that file or it is an invalid YAML file, the 
+will then be restarted with the new flow configuration. If a failure occurs in reading that file or it is an invalid YAML file, the
 update process will be halted.
 
     in minifi.properties to activate the file update trigger specify
@@ -368,19 +368,19 @@ update process will be halted.
 	# specifying a trigger
 	nifi.c2.agent.trigger.classes=FileUpdateTrigger
 	nifi.c2.file.watch=<full path of file to monitor>
-	
 
-	
+
+
 ## Documentation
 
 Type descriptions ( class descriptions entered in PROCESSORS.md ) can be automatically placed within C2 by building cmake with
 the following flag:
 
-	cmake -DBOOTSTRAP=ON .. 
-	
+	cmake -DBOOTSTRAP=ON ..
+
 	You can then run ./extensions/bootstrap/bstrp --inputc2docs <PROCESSORS.md> --outputc2docs ../libminifi/include/agent/agent_docs.h
-	
+
  When cmake is instantiated with this, a build will re-generate the type descriptions from PROCESSORS.md. Once this is finished
- you may re-build the project with the following command from the build directory, running the build as you normally would: 
+ you may re-build the project with the following command from the build directory, running the build as you normally would:
 
 	cmake -DBOOTSTRAP= ..
diff --git a/controller/Controller.h b/controller/Controller.h
index b31bd14..0254d8e 100644
--- a/controller/Controller.h
+++ b/controller/Controller.h
@@ -315,12 +315,11 @@ void printManifest(const std::shared_ptr<org::apache::nifi::minifi::Configure> &
     org::apache::nifi::minifi::setDefaultDirectory(content_repo_path);
   }
 
-  configuration->set("nifi.c2.agent.heartbeat.period", "25");
-  configuration->set("nifi.c2.root.classes", "AgentInformation");
-  configuration->set("nifi.c2.enable", "true");
-  configuration->set("nifi.c2.agent.class", "test");
-  configuration->set("c2.agent.listen", "true");
-  configuration->set("nifi.c2.agent.heartbeat.reporter.classes", "AgentPrinter");
+  configuration->set(org::apache::nifi::minifi::Configure::nifi_c2_agent_heartbeat_period, "25");
+  configuration->set(org::apache::nifi::minifi::Configure::nifi_c2_root_classes, "AgentInformation");
+  configuration->set(org::apache::nifi::minifi::Configure::nifi_c2_enable, "true");
+  configuration->set(org::apache::nifi::minifi::Configure::nifi_c2_agent_class, "test");
+  configuration->set(org::apache::nifi::minifi::Configure::nifi_c2_agent_heartbeat_reporter_classes, "AgentPrinter");
 
   configuration->get(org::apache::nifi::minifi::Configure::nifi_configuration_class_name, nifi_configuration_class_name);
 
diff --git a/encrypt-config/ConfigFile.cpp b/encrypt-config/ConfigFile.cpp
index 4175716..9b01746 100644
--- a/encrypt-config/ConfigFile.cpp
+++ b/encrypt-config/ConfigFile.cpp
@@ -23,12 +23,7 @@
 #include <optional>
 
 #include "utils/StringUtils.h"
-
-namespace {
-constexpr std::array<const char*, 2> DEFAULT_SENSITIVE_PROPERTIES{"nifi.security.client.pass.phrase",
-                                                                  "nifi.rest.api.password"};
-constexpr const char* ADDITIONAL_SENSITIVE_PROPS_PROPERTY_NAME = "nifi.sensitive.props.additional.keys";
-}  // namespace
+#include "properties/Configuration.h"
 
 namespace org {
 namespace apache {
@@ -36,6 +31,10 @@ namespace nifi {
 namespace minifi {
 namespace encrypt_config {
 
+constexpr std::array<const char*, 2> DEFAULT_SENSITIVE_PROPERTIES{Configuration::nifi_security_client_pass_phrase,
+                                                                  Configuration::nifi_rest_api_password};
+constexpr const char* ADDITIONAL_SENSITIVE_PROPS_PROPERTY_NAME = Configuration::nifi_sensitive_props_additional_keys;
+
 std::vector<std::string> ConfigFile::getSensitiveProperties() const {
   std::vector<std::string> sensitive_properties(DEFAULT_SENSITIVE_PROPERTIES.begin(), DEFAULT_SENSITIVE_PROPERTIES.end());
   const std::optional<std::string> additional_sensitive_props_list = getValue(ADDITIONAL_SENSITIVE_PROPS_PROPERTY_NAME);
diff --git a/encrypt-config/tests/ConfigFileEncryptorTests.cpp b/encrypt-config/tests/ConfigFileEncryptorTests.cpp
index 4ca6655..3807f31 100644
--- a/encrypt-config/tests/ConfigFileEncryptorTests.cpp
+++ b/encrypt-config/tests/ConfigFileEncryptorTests.cpp
@@ -21,12 +21,14 @@
 #include <string>
 
 #include "ConfigFileEncryptor.h"
+#include "properties/Configuration.h"
 
 #include "TestBase.h"
 #include "Catch.h"
 
 using org::apache::nifi::minifi::encrypt_config::ConfigFile;
 using org::apache::nifi::minifi::encrypt_config::encryptSensitivePropertiesInFile;
+using org::apache::nifi::minifi::Configuration;
 namespace utils = org::apache::nifi::minifi::utils;
 
 namespace {
@@ -76,13 +78,13 @@ TEST_CASE("ConfigFileEncryptor can encrypt the sensitive properties", "[encrypt-
 
   SECTION("default properties") {
     ConfigFile test_file{std::ifstream{"resources/minifi.properties"}};
-    std::string original_password = test_file.getValue("nifi.rest.api.password").value();
+    std::string original_password = test_file.getValue(Configuration::nifi_rest_api_password).value();
 
     uint32_t num_properties_encrypted = encryptSensitivePropertiesInFile(test_file, KEY);
 
     REQUIRE(num_properties_encrypted == 1);
     REQUIRE(test_file.size() == 102);
-    REQUIRE(check_encryption(test_file, "nifi.rest.api.password", original_password.length()));
+    REQUIRE(check_encryption(test_file, Configuration::nifi_rest_api_password, original_password.length()));
 
     SECTION("calling encryptSensitiveProperties a second time does nothing") {
       ConfigFile test_file_copy = test_file;
@@ -94,7 +96,7 @@ TEST_CASE("ConfigFileEncryptor can encrypt the sensitive properties", "[encrypt-
     }
 
     SECTION("if you reset the password, it will get encrypted again") {
-      test_file.update("nifi.rest.api.password", original_password);
+      test_file.update(Configuration::nifi_rest_api_password, original_password);
 
       SECTION("remove the .protected property") {
         int num_lines_removed = test_file.erase("nifi.rest.api.password.protected");
@@ -110,7 +112,7 @@ TEST_CASE("ConfigFileEncryptor can encrypt the sensitive properties", "[encrypt-
       uint32_t num_properties_encrypted = encryptSensitivePropertiesInFile(test_file, KEY);
 
       REQUIRE(num_properties_encrypted == 1);
-      REQUIRE(check_encryption(test_file, "nifi.rest.api.password", original_password.length()));
+      REQUIRE(check_encryption(test_file, Configuration::nifi_rest_api_password, original_password.length()));
     }
   }
 
@@ -118,19 +120,19 @@ TEST_CASE("ConfigFileEncryptor can encrypt the sensitive properties", "[encrypt-
     ConfigFile test_file{std::ifstream{"resources/with-additional-sensitive-props.minifi.properties"}};
     size_t original_file_size = test_file.size();
 
-    std::string original_c2_enable = test_file.getValue("nifi.c2.enable").value();
-    std::string original_flow_config_file = test_file.getValue("nifi.flow.configuration.file").value();
-    std::string original_password = test_file.getValue("nifi.rest.api.password").value();
-    std::string original_pass_phrase = test_file.getValue("nifi.security.client.pass.phrase").value();
+    std::string original_c2_enable = test_file.getValue(Configuration::nifi_c2_enable).value();
+    std::string original_flow_config_file = test_file.getValue(Configuration::nifi_flow_configuration_file).value();
+    std::string original_password = test_file.getValue(Configuration::nifi_rest_api_password).value();
+    std::string original_pass_phrase = test_file.getValue(Configuration::nifi_security_client_pass_phrase).value();
 
     uint32_t num_properties_encrypted = encryptSensitivePropertiesInFile(test_file, KEY);
 
     REQUIRE(num_properties_encrypted == 4);
     REQUIRE(test_file.size() == original_file_size + 4);
 
-    REQUIRE(check_encryption(test_file, "nifi.c2.enable", original_c2_enable.length()));
-    REQUIRE(check_encryption(test_file, "nifi.flow.configuration.file", original_flow_config_file.length()));
-    REQUIRE(check_encryption(test_file, "nifi.rest.api.password", original_password.length()));
-    REQUIRE(check_encryption(test_file, "nifi.security.client.pass.phrase", original_pass_phrase.length()));
+    REQUIRE(check_encryption(test_file, Configuration::nifi_c2_enable, original_c2_enable.length()));
+    REQUIRE(check_encryption(test_file, Configuration::nifi_flow_configuration_file, original_flow_config_file.length()));
+    REQUIRE(check_encryption(test_file, Configuration::nifi_rest_api_password, original_password.length()));
+    REQUIRE(check_encryption(test_file, Configuration::nifi_security_client_pass_phrase, original_pass_phrase.length()));
   }
 }
diff --git a/encrypt-config/tests/ConfigFileTests.cpp b/encrypt-config/tests/ConfigFileTests.cpp
index d257a00..894cd3a 100644
--- a/encrypt-config/tests/ConfigFileTests.cpp
+++ b/encrypt-config/tests/ConfigFileTests.cpp
@@ -21,6 +21,7 @@
 #include <vector>
 
 #include "ConfigFile.h"
+#include "properties/Configuration.h"
 
 #include "utils/gsl.h"
 
@@ -49,6 +50,7 @@ class ConfigFileTestAccessor {
 
 using org::apache::nifi::minifi::encrypt_config::ConfigFile;
 using org::apache::nifi::minifi::encrypt_config::ConfigFileTestAccessor;
+using org::apache::nifi::minifi::Configuration;
 
 TEST_CASE("ConfigLine can be constructed from a line", "[encrypt-config][constructor]") {
   auto line_is_parsed_correctly = [](const std::string& line, const std::string& expected_key, const std::string& expected_value) {
@@ -113,9 +115,9 @@ TEST_CASE("ConfigFile can parse a simple config file", "[encrypt-config][constru
 
 TEST_CASE("ConfigFile can test whether a key is present", "[encrypt-config][hasValue]") {
   ConfigFile test_file{std::ifstream{"resources/minifi.properties"}};
-  REQUIRE(test_file.hasValue("nifi.version"));
-  REQUIRE(test_file.hasValue("nifi.c2.flow.id"));  // present but blank
-  REQUIRE(!test_file.hasValue("nifi.remote.input.secure"));  // commented out
+  REQUIRE(test_file.hasValue(Configuration::nifi_version));
+  REQUIRE(test_file.hasValue(Configuration::nifi_c2_flow_id));  // present but blank
+  REQUIRE(!test_file.hasValue(Configuration::nifi_remote_input_secure));  // commented out
   REQUIRE(!test_file.hasValue("nifi.this.property.does.not.exist"));
 }
 
@@ -123,11 +125,11 @@ TEST_CASE("ConfigFile can read empty properties correctly", "[encrypt-config][co
   ConfigFile test_file{std::ifstream{"resources/with-additional-sensitive-props.minifi.properties"}};
   REQUIRE(test_file.size() == 103);
 
-  auto empty_property = test_file.getValue("nifi.security.need.ClientAuth");
+  auto empty_property = test_file.getValue(Configuration::nifi_security_need_ClientAuth);
   REQUIRE(empty_property);
   REQUIRE(empty_property->empty());
 
-  auto whitespace_property = test_file.getValue("nifi.security.client.certificate");  // value = " \t\r"
+  auto whitespace_property = test_file.getValue(Configuration::nifi_security_client_certificate);  // value = " \t\r"
   REQUIRE(whitespace_property);
   REQUIRE(whitespace_property->empty());
 }
@@ -136,7 +138,7 @@ TEST_CASE("ConfigFile can find the value for a key", "[encrypt-config][getValue]
   ConfigFile test_file{std::ifstream{"resources/minifi.properties"}};
 
   SECTION("valid key") {
-    REQUIRE(test_file.getValue("nifi.bored.yield.duration") == "10 millis");
+    REQUIRE(test_file.getValue(Configuration::nifi_bored_yield_duration) == "10 millis");
   }
 
   SECTION("nonexistent key") {
@@ -148,8 +150,8 @@ TEST_CASE("ConfigFile can update the value for a key", "[encrypt-config][update]
   ConfigFile test_file{std::ifstream{"resources/minifi.properties"}};
 
   SECTION("valid key") {
-    test_file.update("nifi.bored.yield.duration", "20 millis");
-    REQUIRE(test_file.getValue("nifi.bored.yield.duration") == "20 millis");
+    test_file.update(Configuration::nifi_bored_yield_duration, "20 millis");
+    REQUIRE(test_file.getValue(Configuration::nifi_bored_yield_duration) == "20 millis");
   }
 
   SECTION("nonexistent key") {
@@ -161,7 +163,7 @@ TEST_CASE("ConfigFile can add a new setting after an existing setting", "[encryp
   ConfigFile test_file{std::ifstream{"resources/minifi.properties"}};
 
   SECTION("valid key") {
-    test_file.insertAfter("nifi.rest.api.password", "nifi.rest.api.password.protected", "my-cipher-name");
+    test_file.insertAfter(Configuration::nifi_rest_api_password, "nifi.rest.api.password.protected", "my-cipher-name");
     REQUIRE(test_file.size() == 102);
     REQUIRE(test_file.getValue("nifi.rest.api.password.protected") == "my-cipher-name");
   }
@@ -183,7 +185,7 @@ TEST_CASE("ConfigFile can add a new setting at the end", "[encrypt-config][appen
 
 TEST_CASE("ConfigFile can write to a new file", "[encrypt-config][writeTo]") {
   ConfigFile test_file{std::ifstream{"resources/minifi.properties"}};
-  test_file.update("nifi.bored.yield.duration", "20 millis");
+  test_file.update(Configuration::nifi_bored_yield_duration, "20 millis");
 
   TestController test_controller;
   std::string temp_dir = test_controller.createTempDirectory();
@@ -194,7 +196,7 @@ TEST_CASE("ConfigFile can write to a new file", "[encrypt-config][writeTo]") {
 
   ConfigFile test_file_copy{std::ifstream{file_path}};
   REQUIRE(test_file.size() == test_file_copy.size());
-  REQUIRE(test_file_copy.getValue("nifi.bored.yield.duration") == "20 millis");
+  REQUIRE(test_file_copy.getValue(Configuration::nifi_bored_yield_duration) == "20 millis");
 }
 
 TEST_CASE("ConfigFile will throw if we try to write to an invalid file name", "[encrypt-config][writeTo]") {
@@ -230,14 +232,15 @@ TEST_CASE("ConfigFile can merge lists of property names", "[encrypt-config][merg
 TEST_CASE("ConfigFile can find the list of sensitive properties", "[encrypt-config][getSensitiveProperties]") {
   SECTION("default properties") {
     ConfigFile test_file{std::ifstream{"resources/minifi.properties"}};
-    std::vector<std::string> expected_properties{"nifi.rest.api.password"};
+    std::vector<std::string> expected_properties{Configuration::nifi_rest_api_password};
     REQUIRE(test_file.getSensitiveProperties() == expected_properties);
   }
 
   SECTION("with additional properties") {
     ConfigFile test_file{std::ifstream{"resources/with-additional-sensitive-props.minifi.properties"}};
     std::vector<std::string> expected_properties{
-        "nifi.c2.enable", "nifi.flow.configuration.file", "nifi.rest.api.password", "nifi.security.client.pass.phrase"};
+        Configuration::nifi_c2_enable, Configuration::nifi_flow_configuration_file,
+        Configuration::nifi_rest_api_password, Configuration::nifi_security_client_pass_phrase};
     REQUIRE(test_file.getSensitiveProperties() == expected_properties);
   }
 }
diff --git a/extensions/civetweb/protocols/RESTReceiver.cpp b/extensions/civetweb/protocols/RESTReceiver.cpp
index a3ecc1b..9523eb8 100644
--- a/extensions/civetweb/protocols/RESTReceiver.cpp
+++ b/extensions/civetweb/protocols/RESTReceiver.cpp
@@ -26,6 +26,7 @@
 #include <vector>
 
 #include "core/Resource.h"
+#include "properties/Configuration.h"
 
 namespace org {
 namespace apache {
@@ -51,8 +52,8 @@ void RESTReceiver::initialize(core::controller::ControllerServiceProvider* contr
   logger_->log_trace("Initializing rest receiver");
   if (nullptr != configuration_) {
     std::string listeningPort, rootUri = "/", caCert;
-    configuration_->get("nifi.c2.rest.listener.port", "c2.rest.listener.port", listeningPort);
-    configuration_->get("nifi.c2.rest.listener.cacert", "c2.rest.listener.cacert", caCert);
+    configuration_->get(Configuration::nifi_c2_rest_listener_port, "c2.rest.listener.port", listeningPort);
+    configuration_->get(Configuration::nifi_c2_rest_listener_cacert, "c2.rest.listener.cacert", caCert);
     if (!listeningPort.empty() && !rootUri.empty()) {
       handler = std::make_unique<ListeningProtocol>();
       if (!caCert.empty()) {
diff --git a/extensions/coap/controllerservice/CoapConnector.cpp b/extensions/coap/controllerservice/CoapConnector.cpp
index da6fc03..3840a6a 100644
--- a/extensions/coap/controllerservice/CoapConnector.cpp
+++ b/extensions/coap/controllerservice/CoapConnector.cpp
@@ -67,7 +67,7 @@ void CoapConnectorService::onEnable() {
     core::Property::StringToInt(port_str, port_);
   } else {
     // this is the case where we aren't being used in the context of a single controller service.
-    if (configuration_->get("nifi.c2.agent.coap.host", host_) && configuration_->get("nifi.c2.agent.coap.port", port_str)) {
+    if (configuration_->get(Configuration::nifi_c2_agent_coap_host, host_) && configuration_->get(Configuration::nifi_c2_agent_coap_port, port_str)) {
       core::Property::StringToInt(port_str, port_);
     }
   }
diff --git a/extensions/coap/protocols/CoapC2Protocol.cpp b/extensions/coap/protocols/CoapC2Protocol.cpp
index 0f52b65..1d1f396 100644
--- a/extensions/coap/protocols/CoapC2Protocol.cpp
+++ b/extensions/coap/protocols/CoapC2Protocol.cpp
@@ -41,7 +41,7 @@ CoapProtocol::~CoapProtocol() = default;
 
 void CoapProtocol::initialize(core::controller::ControllerServiceProvider* controller, const std::shared_ptr<Configure> &configure) {
   RESTSender::initialize(controller, configure);
-  if (configure->get("nifi.c2.coap.connector.service", controller_service_name_)) {
+  if (configure->get(minifi::Configuration::nifi_c2_coap_connector_service, controller_service_name_)) {
     auto service = controller->getControllerService(controller_service_name_);
     coap_service_ = std::static_pointer_cast<coap::controllers::CoapConnectorService>(service);
   } else {
diff --git a/extensions/coap/tests/CoapC2VerifyHeartbeat.cpp b/extensions/coap/tests/CoapC2VerifyHeartbeat.cpp
index 4d587cb..0d2ae27 100644
--- a/extensions/coap/tests/CoapC2VerifyHeartbeat.cpp
+++ b/extensions/coap/tests/CoapC2VerifyHeartbeat.cpp
@@ -157,14 +157,14 @@ class VerifyCoAPServer : public CoapIntegrationBase {
       return response;
     });
     server->start();
-    configuration->set("c2.enable", "true");
-    configuration->set("c2.agent.class", "test");
-    configuration->set("nifi.c2.root.classes", "DeviceInfoNode,AgentInformation,FlowInformation,RepositoryMetrics");
-    configuration->set("nifi.c2.agent.protocol.class", "CoapProtocol");
-    configuration->set("nifi.c2.agent.coap.host", "127.0.0.1");
-    configuration->set("nifi.c2.agent.coap.port", new_port_str);
-    configuration->set("c2.agent.heartbeat.period", "10");
-    configuration->set("c2.rest.listener.heartbeat.rooturi", path);
+    configuration->set(minifi::Configuration::nifi_c2_enable, "true");
+    configuration->set(minifi::Configuration::nifi_c2_agent_class, "test");
+    configuration->set(minifi::Configuration::nifi_c2_root_classes, "DeviceInfoNode,AgentInformation,FlowInformation,RepositoryMetrics");
+    configuration->set(minifi::Configuration::nifi_c2_agent_protocol_class, "CoapProtocol");
+    configuration->set(minifi::Configuration::nifi_c2_agent_coap_host, "127.0.0.1");
+    configuration->set(minifi::Configuration::nifi_c2_agent_coap_port, new_port_str);
+    configuration->set(minifi::Configuration::nifi_c2_agent_heartbeat_period, "10");
+    configuration->set(minifi::Configuration::nifi_c2_rest_listener_heartbeat_rooturi, path);
   }
 
  protected:
diff --git a/extensions/coap/tests/CoapIntegrationBase.h b/extensions/coap/tests/CoapIntegrationBase.h
index 071f09f..74f7c34 100644
--- a/extensions/coap/tests/CoapIntegrationBase.h
+++ b/extensions/coap/tests/CoapIntegrationBase.h
@@ -59,7 +59,7 @@ class CoapIntegrationBase : public IntegrationBase {
     if (test_file_location) {
       configuration->set(minifi::Configure::nifi_flow_configuration_file, *test_file_location);
     }
-    configuration->set("c2.agent.heartbeat.period", "200");
+    configuration->set(minifi::Configure::nifi_c2_agent_heartbeat_period, "200");
 
     std::shared_ptr<core::ContentRepository> content_repo = std::make_shared<core::repository::VolatileContentRepository>();
     content_repo->initialize(configuration);
diff --git a/extensions/http-curl/protocols/RESTSender.cpp b/extensions/http-curl/protocols/RESTSender.cpp
index 55cb405..e1dabcb 100644
--- a/extensions/http-curl/protocols/RESTSender.cpp
+++ b/extensions/http-curl/protocols/RESTSender.cpp
@@ -26,6 +26,7 @@
 #include <limits>
 #include "utils/file/FileUtils.h"
 #include "core/Resource.h"
+#include "properties/Configuration.h"
 
 namespace org {
 namespace apache {
@@ -43,9 +44,9 @@ void RESTSender::initialize(core::controller::ControllerServiceProvider* control
   // base URL when one is not specified.
   if (nullptr != configure) {
     std::string update_str, ssl_context_service_str;
-    configure->get("nifi.c2.rest.url", "c2.rest.url", rest_uri_);
-    configure->get("nifi.c2.rest.url.ack", "c2.rest.url.ack", ack_uri_);
-    if (configure->get("nifi.c2.rest.ssl.context.service", "c2.rest.ssl.context.service", ssl_context_service_str)) {
+    configure->get(Configuration::nifi_c2_rest_url, "c2.rest.url", rest_uri_);
+    configure->get(Configuration::nifi_c2_rest_url_ack, "c2.rest.url.ack", ack_uri_);
+    if (configure->get(Configuration::nifi_c2_rest_ssl_context_service, "c2.rest.ssl.context.service", ssl_context_service_str)) {
       auto service = controller->getControllerService(ssl_context_service_str);
       if (nullptr != service) {
         ssl_context_service_ = std::static_pointer_cast<minifi::controllers::SSLContextService>(service);
@@ -74,8 +75,8 @@ C2Payload RESTSender::consumePayload(const C2Payload &payload, Direction directi
 
 void RESTSender::update(const std::shared_ptr<Configure> &configure) {
   std::string url;
-  configure->get("nifi.c2.rest.url", "c2.rest.url", url);
-  configure->get("nifi.c2.rest.url.ack", "c2.rest.url.ack", url);
+  configure->get(Configuration::nifi_c2_rest_url, "c2.rest.url", url);
+  configure->get(Configuration::nifi_c2_rest_url_ack, "c2.rest.url.ack", url);
 }
 
 void RESTSender::setSecurityContext(utils::HTTPClient &client, const std::string &type, const std::string &url) {
diff --git a/extensions/http-curl/tests/AbsoluteTimeoutTest.cpp b/extensions/http-curl/tests/AbsoluteTimeoutTest.cpp
index befaad9..e7f52a6 100644
--- a/extensions/http-curl/tests/AbsoluteTimeoutTest.cpp
+++ b/extensions/http-curl/tests/AbsoluteTimeoutTest.cpp
@@ -16,6 +16,7 @@
  * limitations under the License.
  */
 
+#undef NDEBUG
 #include "TestBase.h"
 #include "Catch.h"
 #include "tests/TestServer.h"
diff --git a/extensions/http-curl/tests/C2DebugBundleTest.cpp b/extensions/http-curl/tests/C2DebugBundleTest.cpp
index e0e4ef6..5af36a5 100644
--- a/extensions/http-curl/tests/C2DebugBundleTest.cpp
+++ b/extensions/http-curl/tests/C2DebugBundleTest.cpp
@@ -28,6 +28,7 @@
 #include "HTTPHandlers.h"
 #include "io/ArchiveStream.h"
 #include "EmptyFlow.h"
+#include "properties/Configuration.h"
 
 using std::literals::chrono_literals::operator""s;
 
@@ -48,7 +49,7 @@ class VerifyDebugInfo : public VerifyC2Base {
 
   void configureC2() override {
     VerifyC2Base::configureC2();
-    configuration->set("nifi.c2.agent.heartbeat.period", "100");
+    configuration->set(org::apache::nifi::minifi::Configuration::nifi_c2_agent_heartbeat_period, "100");
   }
 
   std::function<bool()> verify_;
diff --git a/extensions/http-curl/tests/C2DescribeManifestTest.cpp b/extensions/http-curl/tests/C2DescribeManifestTest.cpp
index b2640f0..9e2c496 100644
--- a/extensions/http-curl/tests/C2DescribeManifestTest.cpp
+++ b/extensions/http-curl/tests/C2DescribeManifestTest.cpp
@@ -30,7 +30,7 @@ class DescribeManifestHandler: public HeartbeatHandler {
   }
 
   void handleAcknowledge(const rapidjson::Document& root) override {
-    verifyJsonHasAgentManifest(root);
+    verifyJsonHasAgentManifest(root, {"InvokeHTTP", "LogAttribute"}, {"nifi.extension.path", "nifi.python.processor.dir"});
   }
 };
 
diff --git a/extensions/http-curl/tests/C2LogHeartbeatTest.cpp b/extensions/http-curl/tests/C2LogHeartbeatTest.cpp
index 6adeed7..09cef80 100644
--- a/extensions/http-curl/tests/C2LogHeartbeatTest.cpp
+++ b/extensions/http-curl/tests/C2LogHeartbeatTest.cpp
@@ -35,6 +35,7 @@
 #include "range/v3/view/transform.hpp"
 #include "utils/IntegrationTestUtils.h"
 #include "utils/StringUtils.h"
+#include "properties/Configuration.h"
 
 class VerifyLogC2Heartbeat : public VerifyC2Base {
  public:
@@ -66,7 +67,7 @@ class VerifyLogC2Heartbeat : public VerifyC2Base {
 
   void configureC2() override {
     VerifyC2Base::configureC2();
-    configuration->set("nifi.c2.agent.heartbeat.reporter.classes", "HeartbeatLogger");
+    configuration->set(org::apache::nifi::minifi::Configuration::nifi_c2_agent_heartbeat_reporter_classes, "HeartbeatLogger");
   }
 };
 
diff --git a/extensions/http-curl/tests/C2NullConfiguration.cpp b/extensions/http-curl/tests/C2NullConfiguration.cpp
index ca12f31..1869a08 100644
--- a/extensions/http-curl/tests/C2NullConfiguration.cpp
+++ b/extensions/http-curl/tests/C2NullConfiguration.cpp
@@ -33,6 +33,8 @@
 #include "HTTPIntegrationBase.h"
 #include "utils/IntegrationTestUtils.h"
 
+namespace org::apache::nifi::minifi {
+
 class VerifyC2Server : public HTTPIntegrationBase {
  public:
   explicit VerifyC2Server(bool isSecure)
@@ -42,11 +44,11 @@ class VerifyC2Server : public HTTPIntegrationBase {
 
   void testSetup() override {
     LogTestController::getInstance().setDebug<utils::HTTPClient>();
-    LogTestController::getInstance().setDebug<minifi::processors::InvokeHTTP>();
-    LogTestController::getInstance().setDebug<minifi::c2::RESTReceiver>();
-    LogTestController::getInstance().setDebug<minifi::c2::C2Agent>();
-    LogTestController::getInstance().setDebug<minifi::processors::LogAttribute>();
-    LogTestController::getInstance().setDebug<minifi::core::ProcessSession>();
+    LogTestController::getInstance().setDebug<processors::InvokeHTTP>();
+    LogTestController::getInstance().setDebug<c2::RESTReceiver>();
+    LogTestController::getInstance().setDebug<c2::C2Agent>();
+    LogTestController::getInstance().setDebug<processors::LogAttribute>();
+    LogTestController::getInstance().setDebug<core::ProcessSession>();
     std::fstream file;
     ss << dir << "/" << "tstFile.ext";
     file.open(ss.str(), std::ios::out);
@@ -55,8 +57,7 @@ class VerifyC2Server : public HTTPIntegrationBase {
   }
 
   void runAssertions() override {
-    using org::apache::nifi::minifi::utils::verifyLogLinePresenceInPollTime;
-    assert(verifyLogLinePresenceInPollTime(std::chrono::milliseconds(wait_time_),
+    assert(utils::verifyLogLinePresenceInPollTime(std::chrono::milliseconds(wait_time_),
         "C2Agent] [error] Could not instantiate null",
         "Class is RESTSender"));
   }
@@ -69,19 +70,19 @@ class VerifyC2Server : public HTTPIntegrationBase {
 
     assert(inv != nullptr);
     std::string url;
-    inv->getProperty(minifi::processors::InvokeHTTP::URL.getName(), url);
+    inv->getProperty(processors::InvokeHTTP::URL.getName(), url);
 
     std::string port, scheme, path;
     parse_http_components(url, port, scheme, path);
-    configuration->set("c2.enable", "true");
-    configuration->set("c2.agent.class", "test");
-    configuration->set("c2.agent.protocol.class", "RESTSender");
-    configuration->set("c2.rest.url", "");
-    configuration->set("c2.rest.url.ack", "");
-    configuration->set("c2.agent.heartbeat.reporter.classes", "null");
-    configuration->set("c2.rest.listener.port", "null");
-    configuration->set("c2.agent.heartbeat.period", "null");
-    configuration->set("c2.rest.listener.heartbeat.rooturi", "null");
+    configuration->set(Configuration::nifi_c2_enable, "true");
+    configuration->set(Configuration::nifi_c2_agent_class, "test");
+    configuration->set(Configuration::nifi_c2_agent_protocol_class, "RESTSender");
+    configuration->set(Configuration::nifi_c2_rest_url, "");
+    configuration->set(Configuration::nifi_c2_rest_url_ack, "");
+    configuration->set(Configuration::nifi_c2_agent_heartbeat_reporter_classes, "null");
+    configuration->set(Configuration::nifi_c2_rest_listener_port, "null");
+    configuration->set(Configuration::nifi_c2_agent_heartbeat_period, "null");
+    configuration->set(Configuration::nifi_c2_rest_listener_heartbeat_rooturi, "null");
   }
 
  protected:
@@ -91,11 +92,13 @@ class VerifyC2Server : public HTTPIntegrationBase {
   TestController testController;
 };
 
+}  // namespace org::apache::nifi::minifi
+
 int main(int argc, char **argv) {
   const cmd_args args = parse_cmdline_args(argc, argv);
   const bool isSecure = args.isUrlSecure();
 
-  VerifyC2Server harness(isSecure);
+  org::apache::nifi::minifi::VerifyC2Server harness(isSecure);
   harness.setKeyDir(args.key_dir);
   harness.run(args.test_file);
   return 0;
diff --git a/extensions/http-curl/tests/C2PauseResumeTest.cpp b/extensions/http-curl/tests/C2PauseResumeTest.cpp
index 905e18d..1314cde 100644
--- a/extensions/http-curl/tests/C2PauseResumeTest.cpp
+++ b/extensions/http-curl/tests/C2PauseResumeTest.cpp
@@ -38,7 +38,7 @@ class VerifyC2PauseResume : public VerifyC2Base {
 
   void configureC2() override {
     VerifyC2Base::configureC2();
-    configuration->set("nifi.c2.agent.heartbeat.period", "500");
+    configuration->set(org::apache::nifi::minifi::Configuration::nifi_c2_agent_heartbeat_period, "500");
   }
 
   void runAssertions() override {
diff --git a/extensions/http-curl/tests/C2RequestClassTest.cpp b/extensions/http-curl/tests/C2RequestClassTest.cpp
index e49fce6..ed97fa0 100644
--- a/extensions/http-curl/tests/C2RequestClassTest.cpp
+++ b/extensions/http-curl/tests/C2RequestClassTest.cpp
@@ -26,6 +26,7 @@
 #include "utils/IntegrationTestUtils.h"
 #include "CivetStream.h"
 #include "StreamPipe.h"
+#include "properties/Configuration.h"
 
 class C2HeartbeatHandler : public ServerAwareHandler {
  public:
@@ -75,10 +76,10 @@ class VerifyC2ClassRequest : public VerifyC2Base {
   explicit VerifyC2ClassRequest(std::function<bool()> verify) : verify_(std::move(verify)) {}
 
   void configureC2() override {
-    configuration->set("nifi.c2.agent.protocol.class", "RESTSender");
-    configuration->set("nifi.c2.enable", "true");
-    configuration->set("nifi.c2.agent.heartbeat.period", "100");
-    configuration->set("nifi.c2.root.classes", "DeviceInfoNode,AgentInformation,FlowInformation");
+    configuration->set(org::apache::nifi::minifi::Configuration::nifi_c2_agent_protocol_class, "RESTSender");
+    configuration->set(org::apache::nifi::minifi::Configuration::nifi_c2_enable, "true");
+    configuration->set(org::apache::nifi::minifi::Configuration::nifi_c2_agent_heartbeat_period, "100");
+    configuration->set(org::apache::nifi::minifi::Configuration::nifi_c2_root_classes, "DeviceInfoNode,AgentInformation,FlowInformation");
   }
 
   void runAssertions() override {
diff --git a/extensions/http-curl/tests/C2VerifyHeartbeatAndStop.cpp b/extensions/http-curl/tests/C2VerifyHeartbeatAndStop.cpp
index a576f02..2fe629f 100644
--- a/extensions/http-curl/tests/C2VerifyHeartbeatAndStop.cpp
+++ b/extensions/http-curl/tests/C2VerifyHeartbeatAndStop.cpp
@@ -26,6 +26,7 @@
 #include "HTTPIntegrationBase.h"
 #include "HTTPHandlers.h"
 #include "utils/IntegrationTestUtils.h"
+#include "properties/Configuration.h"
 
 class LightWeightC2Handler : public HeartbeatHandler {
  public:
@@ -61,14 +62,14 @@ class VerifyC2Heartbeat : public VerifyC2Base {
   }
 
   void configureFullHeartbeat() override {
-    configuration->set("nifi.c2.full.heartbeat", "true");
+    configuration->set(minifi::Configuration::nifi_c2_full_heartbeat, "true");
   }
 };
 
 class VerifyLightWeightC2Heartbeat : public VerifyC2Heartbeat {
  public:
   void configureFullHeartbeat() override {
-    configuration->set("nifi.c2.full.heartbeat", "false");
+    configuration->set(minifi::Configuration::nifi_c2_full_heartbeat, "false");
   }
 };
 
diff --git a/extensions/http-curl/tests/C2VerifyResourceConsumptionInHeartbeat.cpp b/extensions/http-curl/tests/C2VerifyResourceConsumptionInHeartbeat.cpp
index 430afcb..9cae9cf 100644
--- a/extensions/http-curl/tests/C2VerifyResourceConsumptionInHeartbeat.cpp
+++ b/extensions/http-curl/tests/C2VerifyResourceConsumptionInHeartbeat.cpp
@@ -28,6 +28,7 @@
 #include "HTTPIntegrationBase.h"
 #include "HTTPHandlers.h"
 #include "utils/IntegrationTestUtils.h"
+#include "properties/Configuration.h"
 
 class ResourceConsumptionInHeartbeatHandler : public HeartbeatHandler {
  public:
@@ -114,7 +115,7 @@ class VerifyResourceConsumptionInHeartbeat : public VerifyC2Base {
   }
 
   void configureFullHeartbeat() override {
-    configuration->set("nifi.c2.full.heartbeat", "false");
+    configuration->set(minifi::Configuration::nifi_c2_full_heartbeat, "false");
   }
 
   std::function<bool()> event_to_wait_for_;
diff --git a/extensions/http-curl/tests/C2VerifyServeResults.cpp b/extensions/http-curl/tests/C2VerifyServeResults.cpp
index 5763964..b7f4615 100644
--- a/extensions/http-curl/tests/C2VerifyServeResults.cpp
+++ b/extensions/http-curl/tests/C2VerifyServeResults.cpp
@@ -29,6 +29,7 @@
 #include "TestServer.h"
 #include "HTTPIntegrationBase.h"
 #include "utils/IntegrationTestUtils.h"
+#include "properties/Configuration.h"
 
 class VerifyC2Server : public HTTPIntegrationBase {
  public:
@@ -70,13 +71,13 @@ class VerifyC2Server : public HTTPIntegrationBase {
 
     std::string port, scheme, path;
     parse_http_components(url, port, scheme, path);
-    configuration->set("nifi.c2.enable", "true");
-    configuration->set("nifi.c2.agent.class", "test");
-    configuration->set("nifi.c2.agent.heartbeat.reporter.classes", "RESTReceiver");
-    configuration->set("nifi.c2.agent.protocol.class", "RESTSender");
-    configuration->set("nifi.c2.rest.listener.port", port);
-    configuration->set("nifi.c2.agent.heartbeat.period", "10");
-    configuration->set("nifi.c2.rest.listener.heartbeat.rooturi", path);
+    configuration->set(org::apache::nifi::minifi::Configuration::nifi_c2_enable, "true");
+    configuration->set(org::apache::nifi::minifi::Configuration::nifi_c2_agent_class, "test");
+    configuration->set(org::apache::nifi::minifi::Configuration::nifi_c2_agent_heartbeat_reporter_classes, "RESTReceiver");
+    configuration->set(org::apache::nifi::minifi::Configuration::nifi_c2_agent_protocol_class, "RESTSender");
+    configuration->set(org::apache::nifi::minifi::Configuration::nifi_c2_rest_listener_port, port);
+    configuration->set(org::apache::nifi::minifi::Configuration::nifi_c2_agent_heartbeat_period, "10");
+    configuration->set(org::apache::nifi::minifi::Configuration::nifi_c2_rest_listener_heartbeat_rooturi, path);
   }
 
  protected:
diff --git a/extensions/http-curl/tests/HTTPHandlers.h b/extensions/http-curl/tests/HTTPHandlers.h
index 093deca..6f9e193 100644
--- a/extensions/http-curl/tests/HTTPHandlers.h
+++ b/extensions/http-curl/tests/HTTPHandlers.h
@@ -39,6 +39,9 @@
 #include "ServerAwareHandler.h"
 #include "utils/gsl.h"
 #include "agent/build_description.h"
+#include "c2/C2Payload.h"
+#include "properties/Configuration.h"
+#include "range/v3/algorithm/find.hpp"
 
 static std::atomic<int> transaction_id;
 static std::atomic<int> transaction_id_output;
@@ -385,15 +388,20 @@ std::string readPayload(struct mg_connection *conn) {
 
 class HeartbeatHandler : public ServerAwareHandler {
  public:
-  void sendStopOperation(struct mg_connection *conn) {
-    std::string resp = "{\"operation\" : \"heartbeat\", \"requested_operations\" : [{ \"operationid\" : 41, \"operation\" : \"stop\", \"operand\" : \"invoke\"  }, "
-        "{ \"operationid\" : 42, \"operation\" : \"stop\", \"operand\" : \"FlowController\"  } ]}";
-    mg_printf(conn, "HTTP/1.1 200 OK\r\nContent-Type: "
-              "text/plain\r\nContent-Length: %lu\r\nConnection: close\r\n\r\n",
-              resp.length());
-    mg_printf(conn, "%s", resp.c_str());
+  virtual void handleHeartbeat(const rapidjson::Document& root, struct mg_connection *) {
+    verifyJsonHasAgentManifest(root);
+  }
+
+  virtual void handleAcknowledge(const rapidjson::Document&) {
+  }
+
+  bool handlePost(CivetServer *, struct mg_connection *conn) override {
+    verify(conn);
+    sendStopOperation(conn);
+    return true;
   }
 
+ protected:
   void sendHeartbeatResponse(const std::string& operation, const std::string& operand, const std::string& operationId, struct mg_connection * conn,
       const std::unordered_map<std::string, std::string>& args = {}) {
     std::string resp_args;
@@ -421,7 +429,7 @@ class HeartbeatHandler : public ServerAwareHandler {
       mg_printf(conn, "%s", heartbeat_response.c_str());
   }
 
-  void verifyJsonHasAgentManifest(const rapidjson::Document& root) {
+  void verifyJsonHasAgentManifest(const rapidjson::Document& root, const std::vector<std::string>& verify_components = {}, const std::vector<std::string>& disallowed_properties = {}) {
     bool found = false;
     assert(root.HasMember("agentInfo"));
     assert(root["agentInfo"].HasMember("agentManifest"));
@@ -453,13 +461,122 @@ class HeartbeatHandler : public ServerAwareHandler {
     }
     assert(found);
     (void)found;  // unused in release builds
+
+    verifySupportedOperations(root, verify_components, disallowed_properties);
   }
 
-  virtual void handleHeartbeat(const rapidjson::Document& root, struct mg_connection *) {
-    verifyJsonHasAgentManifest(root);
+ private:
+  using Metadata = std::unordered_map<std::string, std::vector<std::unordered_map<std::string, std::string>>>;
+
+  static void sendStopOperation(struct mg_connection *conn) {
+    std::string resp = "{\"operation\" : \"heartbeat\", \"requested_operations\" : [{ \"operationid\" : 41, \"operation\" : \"stop\", \"operand\" : \"invoke\"  }, "
+        "{ \"operationid\" : 42, \"operation\" : \"stop\", \"operand\" : \"FlowController\"  } ]}";
+    mg_printf(conn, "HTTP/1.1 200 OK\r\nContent-Type: "
+              "text/plain\r\nContent-Length: %lu\r\nConnection: close\r\n\r\n",
+              resp.length());
+    mg_printf(conn, "%s", resp.c_str());
   }
 
-  virtual void handleAcknowledge(const rapidjson::Document&) {
+  static std::set<std::string> getOperandsOfProperties(const rapidjson::Value& operation_node) {
+    std::set<std::string> operands;
+    assert(operation_node.HasMember("properties"));
+    const auto& properties_node = operation_node["properties"];
+    for (auto it = properties_node.MemberBegin(); it != properties_node.MemberEnd(); ++it) {
+      operands.insert(it->name.GetString());
+    }
+    return operands;
+  }
+
+  static void verifyMetadata(const rapidjson::Value& operation_node, const std::unordered_map<std::string, Metadata>& operand_with_metadata) {
+    std::unordered_map<std::string, Metadata> operand_with_metadata_found;
+    const auto& properties_node = operation_node["properties"];
+    for (auto prop_it = properties_node.MemberBegin(); prop_it != properties_node.MemberEnd(); ++prop_it) {
+      if (prop_it->value.ObjectEmpty()) {
+        continue;
+      }
+      Metadata metadata_item;
+      for (auto metadata_it = prop_it->value.MemberBegin(); metadata_it != prop_it->value.MemberEnd(); ++metadata_it) {
+        std::vector<std::unordered_map<std::string, std::string>> values;
+        for (const auto& value : metadata_it->value.GetArray()) {
+          std::unordered_map<std::string, std::string> value_item;
+          for (auto value_it = value.MemberBegin(); value_it != value.MemberEnd(); ++value_it) {
+            value_item.emplace(value_it->name.GetString(), value_it->value.GetString());
+          }
+          values.push_back(value_item);
+        }
+        metadata_item.emplace(metadata_it->name.GetString(), values);
+      }
+      operand_with_metadata_found.emplace(prop_it->name.GetString(), metadata_item);
+    }
+    assert(operand_with_metadata_found == operand_with_metadata);
+  }
+
+  template<typename T>
+  void verifyOperands(const rapidjson::Value& operation_node, const std::unordered_map<std::string, Metadata>& operand_with_metadata = {}) {
+    auto operands = getOperandsOfProperties(operation_node);
+    assert(operands == T::values());
+    verifyMetadata(operation_node, operand_with_metadata);
+  }
+
+  void verifyProperties(const rapidjson::Value& operation_node, minifi::c2::Operation operation,
+      const std::vector<std::string>& verify_components, const std::vector<std::string>& disallowed_properties) {
+    switch (operation.value()) {
+      case minifi::c2::Operation::DESCRIBE: {
+        verifyOperands<minifi::c2::DescribeOperand>(operation_node);
+        break;
+      }
+      case minifi::c2::Operation::UPDATE: {
+        std::vector<std::unordered_map<std::string, std::string>> config_properties;
+        for (const auto& property : minifi::Configuration::CONFIGURATION_PROPERTIES) {
+          std::unordered_map<std::string, std::string> config_property;
+          if (ranges::find(disallowed_properties, property.name) == ranges::end(disallowed_properties)) {
+            config_property.emplace("propertyName", property.name);
+            config_property.emplace("validator", property.validator->getName());
+            config_properties.push_back(config_property);
+          }
+        }
+        Metadata metadata;
+        metadata.emplace("availableProperties", config_properties);
+        std::unordered_map<std::string, Metadata> operand_with_metadata;
+        operand_with_metadata.emplace("properties", metadata);
+        verifyOperands<minifi::c2::UpdateOperand>(operation_node, operand_with_metadata);
+        break;
+      }
+      case minifi::c2::Operation::TRANSFER: {
+        verifyOperands<minifi::c2::TransferOperand>(operation_node);
+        break;
+      }
+      case minifi::c2::Operation::CLEAR: {
+        verifyOperands<minifi::c2::ClearOperand>(operation_node);
+        break;
+      }
+      case minifi::c2::Operation::START:
+      case minifi::c2::Operation::STOP: {
+        auto operands = getOperandsOfProperties(operation_node);
+        assert(operands.find("c2") != operands.end());
+        assert(operands.find("FlowController") != operands.end());
+        for (const auto& component : verify_components) {
+          assert(operands.find(component) != operands.end());
+        }
+        break;
+      }
+      default:
+        break;
+    }
+  }
+
+  void verifySupportedOperations(const rapidjson::Document& root, const std::vector<std::string>& verify_components, const std::vector<std::string>& disallowed_properties) {
+    auto& agent_manifest = root["agentInfo"]["agentManifest"];
+    assert(agent_manifest.HasMember("supportedOperations"));
+
+    std::set<std::string> operations;
+    for (const auto& operation_node : agent_manifest["supportedOperations"].GetArray()) {
+      assert(operation_node.HasMember("type"));
+      operations.insert(operation_node["type"].GetString());
+      verifyProperties(operation_node, minifi::c2::Operation::parse(operation_node["type"].GetString()), verify_components, disallowed_properties);
+    }
+
+    assert(operations == minifi::c2::Operation::values());
   }
 
   void verify(struct mg_connection *conn) {
@@ -483,12 +600,6 @@ class HeartbeatHandler : public ServerAwareHandler {
       }
     }
   }
-
-  bool handlePost(CivetServer *, struct mg_connection *conn) override {
-    verify(conn);
-    sendStopOperation(conn);
-    return true;
-  }
 };
 
 class C2FlowProvider : public ServerAwareHandler {
diff --git a/extensions/http-curl/tests/HTTPIntegrationBase.h b/extensions/http-curl/tests/HTTPIntegrationBase.h
index 5e310c1..407399f 100644
--- a/extensions/http-curl/tests/HTTPIntegrationBase.h
+++ b/extensions/http-curl/tests/HTTPIntegrationBase.h
@@ -29,6 +29,7 @@
 #include "Catch.h"
 #include "utils/IntegrationTestUtils.h"
 #include "TestServer.h"
+#include "properties/Configuration.h"
 
 int log_message(const struct mg_connection* /*conn*/, const char *message) {
   puts(message);
@@ -64,7 +65,7 @@ class HTTPIntegrationBase : public IntegrationBase {
 
   std::string getC2RestUrl() const {
     std::string c2_rest_url;
-    configuration->get("nifi.c2.rest.url", c2_rest_url);
+    configuration->get(org::apache::nifi::minifi::Configuration::nifi_c2_rest_url, c2_rest_url);
     return c2_rest_url;
   }
 
@@ -107,8 +108,8 @@ void HTTPIntegrationBase::setUrl(const std::string &url, ServerAwareHandler *han
     }
   }
   std::string c2_url = std::string("http") + (secure ? "s" : "") + "://localhost:" + getWebPort() + url_path;
-  configuration->set("nifi.c2.rest.url", c2_url);
-  configuration->set("nifi.c2.rest.url.ack", c2_url);
+  configuration->set(org::apache::nifi::minifi::Configuration::nifi_c2_rest_url, c2_url);
+  configuration->set(org::apache::nifi::minifi::Configuration::nifi_c2_rest_url_ack, c2_url);
 }
 
 void HTTPIntegrationBase::setC2Url(const std::string &heartbeat_path, const std::string &acknowledge_path) {
@@ -117,8 +118,8 @@ void HTTPIntegrationBase::setC2Url(const std::string &heartbeat_path, const std:
   }
   bool secure = port.back() == 's';
   std::string base = std::string("http") + (secure ? "s" : "") + "://localhost:" + getWebPort();
-  configuration->set("nifi.c2.rest.url", base + heartbeat_path);
-  configuration->set("nifi.c2.rest.url.ack", base + acknowledge_path);
+  configuration->set(org::apache::nifi::minifi::Configuration::nifi_c2_rest_url, base + heartbeat_path);
+  configuration->set(org::apache::nifi::minifi::Configuration::nifi_c2_rest_url_ack, base + acknowledge_path);
 }
 
 class VerifyC2Base : public HTTPIntegrationBase {
@@ -130,11 +131,11 @@ class VerifyC2Base : public HTTPIntegrationBase {
   }
 
   void configureC2() override {
-    configuration->set("nifi.c2.agent.protocol.class", "RESTSender");
-    configuration->set("nifi.c2.enable", "true");
-    configuration->set("nifi.c2.agent.class", "test");
-    configuration->set("nifi.c2.agent.heartbeat.period", "1000");
-    configuration->set("nifi.c2.root.classes", "DeviceInfoNode,AgentInformation,FlowInformation");
+    configuration->set(org::apache::nifi::minifi::Configuration::nifi_c2_agent_protocol_class, "RESTSender");
+    configuration->set(org::apache::nifi::minifi::Configuration::nifi_c2_enable, "true");
+    configuration->set(org::apache::nifi::minifi::Configuration::nifi_c2_agent_class, "test");
+    configuration->set(org::apache::nifi::minifi::Configuration::nifi_c2_agent_heartbeat_period, "1000");
+    configuration->set(org::apache::nifi::minifi::Configuration::nifi_c2_root_classes, "DeviceInfoNode,AgentInformation,FlowInformation");
   }
 
   void cleanup() override {
@@ -153,7 +154,7 @@ class VerifyC2Describe : public VerifyC2Base {
   }
 
   void configureFullHeartbeat() override {
-    configuration->set("nifi.c2.full.heartbeat", "false");
+    configuration->set(org::apache::nifi::minifi::Configuration::nifi_c2_full_heartbeat, "false");
   }
 
   void runAssertions() override {
@@ -177,10 +178,10 @@ class VerifyC2Update : public HTTPIntegrationBase {
   }
 
   void configureC2() override {
-    configuration->set("nifi.c2.agent.protocol.class", "RESTSender");
-    configuration->set("nifi.c2.enable", "true");
-    configuration->set("nifi.c2.agent.class", "test");
-    configuration->set("nifi.c2.agent.heartbeat.period", "1000");
+    configuration->set(org::apache::nifi::minifi::Configuration::nifi_c2_agent_protocol_class, "RESTSender");
+    configuration->set(org::apache::nifi::minifi::Configuration::nifi_c2_enable, "true");
+    configuration->set(org::apache::nifi::minifi::Configuration::nifi_c2_agent_class, "test");
+    configuration->set(org::apache::nifi::minifi::Configuration::nifi_c2_agent_heartbeat_period, "1000");
   }
 
   void cleanup() override {
@@ -206,14 +207,14 @@ class VerifyFlowFetched : public HTTPIntegrationBase {
   }
 
   void configureC2() override {
-    configuration->set("nifi.c2.agent.protocol.class", "RESTSender");
-    configuration->set("nifi.c2.enable", "true");
-    configuration->set("nifi.c2.agent.class", "test");
-    configuration->set("nifi.c2.agent.heartbeat.period", "1000");
+    configuration->set(org::apache::nifi::minifi::Configuration::nifi_c2_agent_protocol_class, "RESTSender");
+    configuration->set(org::apache::nifi::minifi::Configuration::nifi_c2_enable, "true");
+    configuration->set(org::apache::nifi::minifi::Configuration::nifi_c2_agent_class, "test");
+    configuration->set(org::apache::nifi::minifi::Configuration::nifi_c2_agent_heartbeat_period, "1000");
   }
 
   void setFlowUrl(const std::string& url) {
-    configuration->set(minifi::Configure::nifi_c2_flow_url, url);
+    configuration->set(org::apache::nifi::minifi::Configuration::nifi_c2_flow_url, url);
   }
 
   void cleanup() override {
diff --git a/extensions/http-curl/tests/HTTPSiteToSiteTests.cpp b/extensions/http-curl/tests/HTTPSiteToSiteTests.cpp
index f0bd873..2672e6b 100644
--- a/extensions/http-curl/tests/HTTPSiteToSiteTests.cpp
+++ b/extensions/http-curl/tests/HTTPSiteToSiteTests.cpp
@@ -34,6 +34,7 @@
 #include "HTTPIntegrationBase.h"
 #include "HTTPHandlers.h"
 #include "client/HTTPStream.h"
+#include "properties/Configuration.h"
 
 using std::literals::chrono_literals::operator""s;
 
@@ -60,9 +61,9 @@ class SiteToSiteTestHarness : public HTTPIntegrationBase {
     file << "tempFile";
     file.close();
 
-    configuration->set("nifi.c2.enable", "false");
-    configuration->set("nifi.remote.input.http.enabled", "true");
-    configuration->set("nifi.remote.input.socket.port", "8099");
+    configuration->set(org::apache::nifi::minifi::Configuration::nifi_c2_enable, "false");
+    configuration->set(org::apache::nifi::minifi::Configuration::nifi_remote_input_http, "true");
+    configuration->set(org::apache::nifi::minifi::Configuration::nifi_remote_input_socket_port, "8099");
   }
 
   void runAssertions() override {
diff --git a/extensions/http-curl/tests/HttpPostIntegrationTest.cpp b/extensions/http-curl/tests/HttpPostIntegrationTest.cpp
index b8b94d8..eb850f6 100644
--- a/extensions/http-curl/tests/HttpPostIntegrationTest.cpp
+++ b/extensions/http-curl/tests/HttpPostIntegrationTest.cpp
@@ -31,6 +31,7 @@
 #include "FlowController.h"
 #include "HTTPIntegrationBase.h"
 #include "utils/IntegrationTestUtils.h"
+#include "properties/Configuration.h"
 
 using std::literals::chrono_literals::operator""s;
 
@@ -60,8 +61,8 @@ class HttpTestHarness : public HTTPIntegrationBase {
     file.open(ss.str(), std::ios::out);
     file << "tempFile";
     file.close();
-    configuration->set("nifi.flow.engine.threads", "8");
-    configuration->set("nifi.c2.enable", "false");
+    configuration->set(org::apache::nifi::minifi::Configuration::nifi_flow_engine_threads, "8");
+    configuration->set(org::apache::nifi::minifi::Configuration::nifi_c2_enable, "false");
   }
 
   void cleanup() override {
diff --git a/extensions/http-curl/tests/TimeoutHTTPSiteToSiteTests.cpp b/extensions/http-curl/tests/TimeoutHTTPSiteToSiteTests.cpp
index 95b027e..89cf70c 100644
--- a/extensions/http-curl/tests/TimeoutHTTPSiteToSiteTests.cpp
+++ b/extensions/http-curl/tests/TimeoutHTTPSiteToSiteTests.cpp
@@ -35,6 +35,7 @@
 #include "HTTPIntegrationBase.h"
 #include "HTTPHandlers.h"
 #include "client/HTTPStream.h"
+#include "properties/Configuration.h"
 
 using std::literals::chrono_literals::operator""s;
 
@@ -61,9 +62,9 @@ class SiteToSiteTestHarness : public HTTPIntegrationBase {
     file << "tempFile";
     file.close();
 
-    configuration->set("nifi.c2.enable", "false");
-    configuration->set("nifi.remote.input.http.enabled", "true");
-    configuration->set("nifi.remote.input.socket.port", "8099");
+    configuration->set(org::apache::nifi::minifi::Configuration::nifi_c2_enable, "false");
+    configuration->set(org::apache::nifi::minifi::Configuration::nifi_remote_input_http, "true");
+    configuration->set(org::apache::nifi::minifi::Configuration::nifi_remote_input_socket_port, "8099");
   }
 
   void runAssertions() override {
diff --git a/extensions/http-curl/tests/VerifyInvokeHTTP.h b/extensions/http-curl/tests/VerifyInvokeHTTP.h
index b9c2a3d..c4552f9 100644
--- a/extensions/http-curl/tests/VerifyInvokeHTTP.h
+++ b/extensions/http-curl/tests/VerifyInvokeHTTP.h
@@ -84,7 +84,7 @@ class VerifyInvokeHTTP : public HTTPIntegrationBase {
     if (flow_yml_path) {
       configuration->set(minifi::Configure::nifi_flow_configuration_file, *flow_yml_path);
     }
-    configuration->set("c2.agent.heartbeat.period", "200");
+    configuration->set(minifi::Configure::nifi_c2_agent_heartbeat_period, "200");
     std::shared_ptr<core::ContentRepository> content_repo = std::make_shared<core::repository::VolatileContentRepository>();
     content_repo->initialize(configuration);
     std::shared_ptr<minifi::io::StreamFactory> stream_factory = minifi::io::StreamFactory::getInstance(configuration);
diff --git a/extensions/jni/JVMCreator.h b/extensions/jni/JVMCreator.h
index 2120191..9979a18 100644
--- a/extensions/jni/JVMCreator.h
+++ b/extensions/jni/JVMCreator.h
@@ -26,6 +26,7 @@
 #include "utils/file/FileUtils.h"
 #include "core/Core.h"
 #include "core/logging/LoggerConfiguration.h"
+#include "properties/Configuration.h"
 
 namespace org {
 namespace apache {
@@ -62,19 +63,19 @@ class JVMCreator : public minifi::core::CoreComponent {
 
     // assuming we have the options set and can access the JVMCreator
 
-    if (configuration->get("nifi.framework.dir", pathListings)) {
+    if (configuration->get(minifi::Configuration::nifi_framework_dir, pathListings)) {
       std::vector<std::string> paths;
       paths.emplace_back(pathListings);
       configure(paths);
 
-      if (configuration->get("nifi.jvm.options", jvmOptionsStr)) {
+      if (configuration->get(minifi::Configuration::nifi_jvm_options, jvmOptionsStr)) {
         jvm_options_ = utils::StringUtils::split(jvmOptionsStr, ",");
       }
 
       initializeJVM();
     }
     std::string nar_dir, nar_dep, nar_docs;
-    if (loader_ && configuration->get("nifi.nar.directory", nar_dir) && configuration->get("nifi.nar.deploy.directory", nar_dep)) {
+    if (loader_ && configuration->get(minifi::Configuration::nifi_nar_directory, nar_dir) && configuration->get(minifi::Configuration::nifi_nar_deploy_directory, nar_dep)) {
       std::shared_ptr<jni::controllers::JavaControllerService> servicer = std::make_shared<jni::controllers::JavaControllerService>("BaseService");
       servicer->initialize();
       servicer->setProperty(jni::controllers::JavaControllerService::NarDirectory, nar_dir);
diff --git a/extensions/mqtt/protocol/MQTTC2Protocol.cpp b/extensions/mqtt/protocol/MQTTC2Protocol.cpp
index 932c3ef..a9b1045 100644
--- a/extensions/mqtt/protocol/MQTTC2Protocol.cpp
+++ b/extensions/mqtt/protocol/MQTTC2Protocol.cpp
@@ -17,6 +17,7 @@
  */
 #include "MQTTC2Protocol.h"
 #include "core/Resource.h"
+#include "properties/Configuration.h"
 
 namespace org {
 namespace apache {
@@ -31,7 +32,7 @@ MQTTC2Protocol::MQTTC2Protocol(const std::string& name, const utils::Identifier&
 MQTTC2Protocol::~MQTTC2Protocol() = default;
 
 void MQTTC2Protocol::initialize(core::controller::ControllerServiceProvider* controller, const std::shared_ptr<Configure> &configure) {
-  if (configure->get("nifi.c2.mqtt.connector.service", controller_service_name_)) {
+  if (configure->get(minifi::Configuration::nifi_c2_mqtt_connector_service, controller_service_name_)) {
     auto service = controller->getControllerService(controller_service_name_);
     mqtt_service_ = std::static_pointer_cast<controllers::MQTTControllerService>(service);
   } else {
@@ -42,12 +43,12 @@ void MQTTC2Protocol::initialize(core::controller::ControllerServiceProvider* con
 
   std::stringstream outputStream;
   std::string updateTopicOpt, heartbeatTopicOpt;
-  if (configure->get("nifi.c2.mqtt.heartbeat.topic", heartbeatTopicOpt)) {
+  if (configure->get(minifi::Configuration::nifi_c2_mqtt_heartbeat_topic, heartbeatTopicOpt)) {
     heartbeat_topic_ = heartbeatTopicOpt;
   } else {
     heartbeat_topic_ = "heartbeats";  // outputStream.str();
   }
-  if (configure->get("nifi.c2.mqtt.update.topic", updateTopicOpt)) {
+  if (configure->get(minifi::Configuration::nifi_c2_mqtt_update_topic, updateTopicOpt)) {
     update_topic_ = updateTopicOpt;
   } else {
     update_topic_ = "updates";
diff --git a/extensions/script/python/PythonCreator.h b/extensions/script/python/PythonCreator.h
index 32bb3c4..b434d41 100644
--- a/extensions/script/python/PythonCreator.h
+++ b/extensions/script/python/PythonCreator.h
@@ -33,6 +33,7 @@
 #include "utils/file/FileUtils.h"
 #include "utils/StringUtils.h"
 #include "range/v3/algorithm.hpp"
+#include "properties/Configuration.h"
 
 namespace org {
 namespace apache {
@@ -59,7 +60,7 @@ class PythonCreator : public minifi::core::CoreComponent {
     python::PythonScriptEngine::initialize();
 
     auto engine = std::make_shared<python::PythonScriptEngine>();
-    std::optional<std::string> pathListings = configuration ? configuration->get("nifi.python.processor.dir") : std::nullopt;
+    std::optional<std::string> pathListings = configuration ? configuration->get(minifi::Configuration::nifi_python_processor_dir) : std::nullopt;
     if (!pathListings) {
       return;
     }
diff --git a/extensions/standard-processors/tests/integration/SecureSocketGetTCPTest.cpp b/extensions/standard-processors/tests/integration/SecureSocketGetTCPTest.cpp
index 17a8fa0..318e95f 100644
--- a/extensions/standard-processors/tests/integration/SecureSocketGetTCPTest.cpp
+++ b/extensions/standard-processors/tests/integration/SecureSocketGetTCPTest.cpp
@@ -93,16 +93,16 @@ class SecureSocketTest : public IntegrationBase {
 
     assert(inv != nullptr);
     std::string url;
-    configuration->set("nifi.remote.input.secure", "true");
+    configuration->set(org::apache::nifi::minifi::Configuration::nifi_remote_input_secure, "true");
     std::string path = key_dir + "cn.crt.pem";
-    configuration->set("nifi.security.client.certificate", path);
+    configuration->set(org::apache::nifi::minifi::Configuration::nifi_security_client_certificate, path);
     path = key_dir + "cn.ckey.pem";
-    configuration->set("nifi.security.client.private.key", path);
+    configuration->set(org::apache::nifi::minifi::Configuration::nifi_security_client_private_key, path);
     path = key_dir + "cn.pass";
-    configuration->set("nifi.security.client.pass.phrase", path);
+    configuration->set(org::apache::nifi::minifi::Configuration::nifi_security_client_pass_phrase, path);
     path = key_dir + "nifi-cert.pem";
-    configuration->set("nifi.security.client.ca.certificate", path);
-    configuration->set("nifi.c2.enable", "false");
+    configuration->set(org::apache::nifi::minifi::Configuration::nifi_security_client_ca_certificate, path);
+    configuration->set(org::apache::nifi::minifi::Configuration::nifi_c2_enable, "false");
     std::string endpoint;
     inv->getProperty(minifi::processors::GetTCP::EndpointList.getName(), endpoint);
     auto endpoints = utils::StringUtils::split(endpoint, ",");
diff --git a/libminifi/include/c2/C2Payload.h b/libminifi/include/c2/C2Payload.h
index d43da4e..94666ee 100644
--- a/libminifi/include/c2/C2Payload.h
+++ b/libminifi/include/c2/C2Payload.h
@@ -44,13 +44,36 @@ SMART_ENUM(Operation,
   (DESCRIBE, "describe"),
   (HEARTBEAT, "heartbeat"),
   (UPDATE, "update"),
-  (VALIDATE, "validate"),
   (CLEAR, "clear"),
   (TRANSFER, "transfer"),
   (PAUSE, "pause"),
   (RESUME, "resume")
 )
 
+SMART_ENUM(DescribeOperand,
+  (METRICS, "metrics"),
+  (CONFIGURATION, "configuration"),
+  (MANIFEST, "manifest"),
+  (JSTACK, "jstack"),
+  (CORECOMPONENTSTATE, "corecomponentstate")
+)
+
+SMART_ENUM(UpdateOperand,
+  (CONFIGURATION, "configuration"),
+  (PROPERTIES, "properties"),
+  (C2, "c2")
+)
+
+SMART_ENUM(TransferOperand,
+  (DEBUG, "debug")
+)
+
+SMART_ENUM(ClearOperand,
+  (CONNECTION, "connection"),
+  (REPOSITORIES, "repositories"),
+  (CORECOMPONENTSTATE, "corecomponentstate")
+)
+
 #define PAYLOAD_NO_STATUS 0
 #define PAYLOAD_SUCCESS 1
 #define PAYLOAD_FAILURE 2
diff --git a/libminifi/include/core/ProcessContext.h b/libminifi/include/core/ProcessContext.h
index 317dca5..1423bc4 100644
--- a/libminifi/include/core/ProcessContext.h
+++ b/libminifi/include/core/ProcessContext.h
@@ -207,7 +207,7 @@ class ProcessContext : public controller::ControllerServiceLookup, public core::
    * @return the ControllerService that is registered with the given
    * identifier
    */
-  std::shared_ptr<core::controller::ControllerService> getControllerService(const std::string &identifier) {
+  std::shared_ptr<core::controller::ControllerService> getControllerService(const std::string &identifier) const override {
     return controller_service_provider_ == nullptr ? nullptr : controller_service_provider_->getControllerServiceForComponent(identifier, processor_node_->getUUID());
   }
 
@@ -218,7 +218,7 @@ class ProcessContext : public controller::ControllerServiceLookup, public core::
    * identifier is not known by this ControllerServiceLookup, returns
    * <code>false</code>
    */
-  bool isControllerServiceEnabled(const std::string &identifier) {
+  bool isControllerServiceEnabled(const std::string &identifier) override {
     return controller_service_provider_->isControllerServiceEnabled(identifier);
   }
 
@@ -229,7 +229,7 @@ class ProcessContext : public controller::ControllerServiceLookup, public core::
    * otherwise returns <code>false</code>. If the given identifier is not
    * known by this ControllerServiceLookup, returns <code>false</code>
    */
-  bool isControllerServiceEnabling(const std::string &identifier) {
+  bool isControllerServiceEnabling(const std::string &identifier) override {
     return controller_service_provider_->isControllerServiceEnabling(identifier);
   }
 
@@ -238,7 +238,7 @@ class ProcessContext : public controller::ControllerServiceLookup, public core::
    * @return the name of the Controller service with the given identifier. If
    * no service can be found with this identifier, returns {@code null}
    */
-  const std::string getControllerServiceName(const std::string &identifier) {
+  const std::string getControllerServiceName(const std::string &identifier) const override {
     return controller_service_provider_->getControllerServiceName(identifier);
   }
 
diff --git a/libminifi/include/core/Property.h b/libminifi/include/core/Property.h
index ef151cf..00e6dd1 100644
--- a/libminifi/include/core/Property.h
+++ b/libminifi/include/core/Property.h
@@ -31,6 +31,7 @@
 #include <typeindex>
 #include <utility>
 #include <vector>
+#include <string_view>
 
 #include "CachedValueValidator.h"
 #include "core/Core.h"
@@ -467,6 +468,16 @@ class ConstrainedProperty : public std::enable_shared_from_this<ConstrainedPrope
   friend class PropertyBuilder;
 };
 
+struct ConfigurationProperty {
+  explicit ConfigurationProperty(std::string_view name, gsl::not_null<PropertyValidator*> validator = gsl::make_not_null(StandardValidators::get().VALID_VALIDATOR.get()))
+    : name(name),
+      validator(validator) {
+  }
+
+  std::string_view name;
+  gsl::not_null<PropertyValidator*> validator;
+};
+
 }  // namespace core
 }  // namespace minifi
 }  // namespace nifi
diff --git a/libminifi/include/core/controller/ControllerServiceLookup.h b/libminifi/include/core/controller/ControllerServiceLookup.h
index 73cc8da..54907f6 100644
--- a/libminifi/include/core/controller/ControllerServiceLookup.h
+++ b/libminifi/include/core/controller/ControllerServiceLookup.h
@@ -50,7 +50,7 @@ class ControllerServiceLookup {
    * @param identifier reference string for controller service.
    * @return controller service reference.
    */
-  virtual std::shared_ptr<ControllerService> getControllerService(const std::string &identifier) = 0;
+  virtual std::shared_ptr<ControllerService> getControllerService(const std::string &identifier) const = 0;
 
   /**
    * Detects if controller service is enabled.
@@ -70,7 +70,7 @@ class ControllerServiceLookup {
    * Gets the controller service name for the provided reference identifier
    * @param identifier reference string for the controller service.
    */
-  virtual const std::string getControllerServiceName(const std::string &identifier) = 0;
+  virtual const std::string getControllerServiceName(const std::string &identifier) const = 0;
 };
 
 }  // namespace controller
diff --git a/libminifi/include/core/controller/ControllerServiceProvider.h b/libminifi/include/core/controller/ControllerServiceProvider.h
index 894021d..3cb6edc 100644
--- a/libminifi/include/core/controller/ControllerServiceProvider.h
+++ b/libminifi/include/core/controller/ControllerServiceProvider.h
@@ -175,7 +175,7 @@ class ControllerServiceProvider : public CoreComponent, public ConfigurableCompo
    * Returns a controller service for the service identifier and componentID
    * @param service Identifier service identifier.
    */
-  virtual std::shared_ptr<ControllerService> getControllerServiceForComponent(const std::string& serviceIdentifier, const utils::Identifier& /*componentId*/) {
+  virtual std::shared_ptr<ControllerService> getControllerServiceForComponent(const std::string& serviceIdentifier, const utils::Identifier& /*componentId*/) const {
     std::shared_ptr<ControllerService> node = getControllerService(serviceIdentifier);
     return node;
   }
@@ -184,13 +184,13 @@ class ControllerServiceProvider : public CoreComponent, public ConfigurableCompo
    * Gets the controller service for the provided identifier
    * @param identifier service identifier.
    */
-  virtual std::shared_ptr<ControllerService> getControllerService(const std::string &identifier);
+  std::shared_ptr<ControllerService> getControllerService(const std::string &identifier) const override;
 
   /**
    * Determines if Controller service is enabled.
    * @param identifier service identifier.
    */
-  virtual bool isControllerServiceEnabled(const std::string &identifier) {
+  bool isControllerServiceEnabled(const std::string &identifier) override {
     std::shared_ptr<ControllerServiceNode> node = getControllerServiceNode(identifier);
     if (nullptr != node) {
       return linkedServicesAre(ENABLED, node);
@@ -203,7 +203,7 @@ class ControllerServiceProvider : public CoreComponent, public ConfigurableCompo
    * Determines if Controller service is being enabled.
    * @param identifier service identifier.
    */
-  virtual bool isControllerServiceEnabling(const std::string &identifier) {
+  bool isControllerServiceEnabling(const std::string &identifier) override {
     std::shared_ptr<ControllerServiceNode> node = getControllerServiceNode(identifier);
     if (nullptr != node) {
       return linkedServicesAre(ENABLING, node);
@@ -212,7 +212,7 @@ class ControllerServiceProvider : public CoreComponent, public ConfigurableCompo
     }
   }
 
-  virtual const std::string getControllerServiceName(const std::string &identifier) {
+  const std::string getControllerServiceName(const std::string &identifier) const override {
     std::shared_ptr<ControllerService> node = getControllerService(identifier);
     if (nullptr != node) {
       return node->getName();
@@ -246,7 +246,7 @@ class ControllerServiceProvider : public CoreComponent, public ConfigurableCompo
     }
   }
 
-  bool canEdit() {
+  bool canEdit() override {
     return true;
   }
 
diff --git a/libminifi/include/core/controller/ForwardingControllerServiceProvider.h b/libminifi/include/core/controller/ForwardingControllerServiceProvider.h
index 9aa4114..960307b 100644
--- a/libminifi/include/core/controller/ForwardingControllerServiceProvider.h
+++ b/libminifi/include/core/controller/ForwardingControllerServiceProvider.h
@@ -64,7 +64,7 @@ class ForwardingControllerServiceProvider : public ControllerServiceProvider {
     return controller_service_provider_impl_->clearControllerServices();
   }
 
-  std::shared_ptr<ControllerService> getControllerService(const std::string &identifier) override {
+  std::shared_ptr<ControllerService> getControllerService(const std::string &identifier) const override {
     return controller_service_provider_impl_->getControllerService(identifier);
   }
 
@@ -100,7 +100,7 @@ class ForwardingControllerServiceProvider : public ControllerServiceProvider {
     return controller_service_provider_impl_->scheduleReferencingComponents(serviceNode);
   }
 
-  std::shared_ptr<ControllerService> getControllerServiceForComponent(const std::string &serviceIdentifier, const utils::Identifier &componentId) override {
+  std::shared_ptr<ControllerService> getControllerServiceForComponent(const std::string &serviceIdentifier, const utils::Identifier &componentId) const override {
     return controller_service_provider_impl_->getControllerServiceForComponent(serviceIdentifier, componentId);
   }
 
@@ -112,7 +112,7 @@ class ForwardingControllerServiceProvider : public ControllerServiceProvider {
     return controller_service_provider_impl_->isControllerServiceEnabling(identifier);
   }
 
-  const std::string getControllerServiceName(const std::string &identifier) override {
+  const std::string getControllerServiceName(const std::string &identifier) const override {
     return controller_service_provider_impl_->getControllerServiceName(identifier);
   }
 
diff --git a/libminifi/include/core/extension/ExtensionManager.h b/libminifi/include/core/extension/ExtensionManager.h
index 930dc12..0c099c3 100644
--- a/libminifi/include/core/extension/ExtensionManager.h
+++ b/libminifi/include/core/extension/ExtensionManager.h
@@ -32,12 +32,6 @@ namespace minifi {
 namespace core {
 namespace extension {
 
-/**
- * Comma separated list of path patterns. Patterns prepended with "!" result in the exclusion
- * of the extensions matching that pattern, unless some subsequent pattern re-enables it.
- */
-static constexpr const char* nifi_extension_path = "nifi.extension.path";
-
 class ExtensionManager {
   ExtensionManager();
 
diff --git a/libminifi/include/core/state/Value.h b/libminifi/include/core/state/Value.h
index 8aa2702..b728e6f 100644
--- a/libminifi/include/core/state/Value.h
+++ b/libminifi/include/core/state/Value.h
@@ -555,6 +555,7 @@ struct SerializedResponseNode {
   ValueNode value;
   bool array;
   bool collapsible;
+  bool keep_empty = false;
   std::vector<SerializedResponseNode> children;
 
   SerializedResponseNode(bool collapsible = true) // NOLINT
diff --git a/libminifi/include/core/state/nodes/AgentInformation.h b/libminifi/include/core/state/nodes/AgentInformation.h
index fb54e2e..1f5669d 100644
--- a/libminifi/include/core/state/nodes/AgentInformation.h
+++ b/libminifi/include/core/state/nodes/AgentInformation.h
@@ -62,6 +62,7 @@
 #include "utils/ProcessCpuUsageTracker.h"
 #include "core/AgentIdentificationProvider.h"
 #include "utils/Export.h"
+#include "SupportedOperations.h"
 
 namespace org {
 namespace apache {
@@ -592,7 +593,7 @@ class AgentMonitor {
 
  protected:
   std::map<std::string, std::shared_ptr<core::Repository>> repositories_;
-  state::StateMonitor* monitor_;
+  state::StateMonitor* monitor_ = nullptr;
 };
 
 /**
@@ -600,14 +601,26 @@ class AgentMonitor {
  */
 class AgentManifest : public DeviceInformation {
  public:
+  AgentManifest(std::string name, const utils::Identifier& uuid)
+    : DeviceInformation(std::move(name), uuid) {
+  }
+
   explicit AgentManifest(std::string name)
-      : DeviceInformation(std::move(name)) {
+    : DeviceInformation(std::move(name)) {
   }
 
   std::string getName() const {
     return "agentManifest";
   }
 
+  void setStateMonitor(state::StateMonitor* monitor) {
+    monitor_ = monitor;
+  }
+
+  void setUpdatePolicyController(controllers::UpdatePolicyControllerService* update_policy_controller) {
+    update_policy_controller_ = update_policy_controller;
+  }
+
   std::vector<SerializedResponseNode> serialize() {
     static std::vector<SerializedResponseNode> serialized;
     if (serialized.empty()) {
@@ -671,9 +684,20 @@ class AgentManifest : public DeviceInformation {
       for (auto defaultNode : defaults.serialize()) {
         serialized.push_back(defaultNode);
       }
+
+      SupportedOperations supported_operations("supportedOperations");
+      supported_operations.setStateMonitor(monitor_);
+      supported_operations.setUpdatePolicyController(update_policy_controller_);
+      for (const auto& operation : supported_operations.serialize()) {
+        serialized.push_back(operation);
+      }
     }
     return serialized;
   }
+
+ private:
+  state::StateMonitor* monitor_ = nullptr;
+  controllers::UpdatePolicyControllerService* update_policy_controller_ = nullptr;
 };
 
 class AgentNode : public DeviceInformation, public AgentMonitor, public AgentIdentifier {
@@ -688,6 +712,10 @@ class AgentNode : public DeviceInformation, public AgentMonitor, public AgentIde
     setArray(false);
   }
 
+  void setUpdatePolicyController(controllers::UpdatePolicyControllerService* update_policy_controller) {
+    update_policy_controller_ = update_policy_controller;
+  }
+
  protected:
   std::vector<SerializedResponseNode> serialize() {
     std::vector<SerializedResponseNode> serialized;
@@ -717,7 +745,10 @@ class AgentNode : public DeviceInformation, public AgentMonitor, public AgentIde
   std::vector<SerializedResponseNode> getAgentManifest() const {
     SerializedResponseNode agentManifest;
     agentManifest.name = "agentManifest";
-    agentManifest.children = AgentManifest{"manifest"}.serialize();
+    AgentManifest manifest{"manifest"};
+    manifest.setStateMonitor(monitor_);
+    manifest.setUpdatePolicyController(update_policy_controller_);
+    agentManifest.children = manifest.serialize();
     return std::vector<SerializedResponseNode>{ agentManifest };
   }
 
@@ -748,6 +779,7 @@ class AgentNode : public DeviceInformation, public AgentMonitor, public AgentIde
 
  private:
   std::optional<std::string> agentManifestHash_;
+  controllers::UpdatePolicyControllerService* update_policy_controller_ = nullptr;
 };
 
 /**
diff --git a/libminifi/include/core/state/nodes/SupportedOperations.h b/libminifi/include/core/state/nodes/SupportedOperations.h
new file mode 100644
index 0000000..0291811
--- /dev/null
+++ b/libminifi/include/core/state/nodes/SupportedOperations.h
@@ -0,0 +1,70 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#pragma once
+
+#include <string>
+#include <vector>
+#include <memory>
+#include <unordered_map>
+#include <utility>
+
+#include "MetricsBase.h"
+#include "c2/C2Payload.h"
+#include "controllers/UpdatePolicyControllerService.h"
+
+namespace org::apache::nifi::minifi::state::response {
+
+class SupportedOperations : public DeviceInformation {
+ public:
+  SupportedOperations(const std::string &name, const utils::Identifier &uuid);
+  explicit SupportedOperations(const std::string &name);
+
+  std::string getName() const override;
+  std::vector<SerializedResponseNode> serialize() override;
+  void setStateMonitor(state::StateMonitor* monitor) {
+    monitor_ = monitor;
+  }
+
+  void setUpdatePolicyController(controllers::UpdatePolicyControllerService* update_policy_controller) {
+    update_policy_controller_ = update_policy_controller;
+  }
+
+ private:
+  using Metadata = std::unordered_map<std::string, std::vector<std::unordered_map<std::string, std::string>>>;
+
+  template<typename T>
+  static void serializeProperty(SerializedResponseNode& properties, const std::unordered_map<std::string, Metadata>& operand_with_metadata = {}) {
+    for (const auto& operand_type: T::values()) {
+      auto metadata_it = operand_with_metadata.find(operand_type);
+      if (metadata_it != operand_with_metadata.end()) {
+        addProperty(properties, operand_type, metadata_it->second);
+      } else {
+        addProperty(properties, operand_type);
+      }
+    }
+  }
+
+  static void addProperty(SerializedResponseNode& properties, const std::string& operand, const Metadata& metadata = {});
+  void fillProperties(SerializedResponseNode& properties, minifi::c2::Operation operation) const;
+  Metadata buildUpdatePropertiesMetadata() const;
+
+  state::StateMonitor* monitor_ = nullptr;
+  controllers::UpdatePolicyControllerService* update_policy_controller_ = nullptr;
+};
+
+}  // namespace org::apache::nifi::minifi::state::response
diff --git a/libminifi/include/properties/Configuration.h b/libminifi/include/properties/Configuration.h
index 67361d9..840953e 100644
--- a/libminifi/include/properties/Configuration.h
+++ b/libminifi/include/properties/Configuration.h
@@ -17,22 +17,30 @@
 
 #pragma once
 
-#include <string>
 #include <mutex>
+#include <vector>
+
 #include "properties/Properties.h"
 #include "utils/OptionalUtils.h"
+#include "utils/Export.h"
 
 namespace org {
 namespace apache {
 namespace nifi {
 namespace minifi {
 
-// TODO(adebreceni): eliminate this class in a separate PR
+namespace core {
+  struct ConfigurationProperty;
+}
+
 class Configuration : public Properties {
  public:
   Configuration() : Properties("MiNiFi configuration") {}
 
+  static constexpr const char *nifi_volatile_repository_options = "nifi.volatile.repository.options.";
+
   // nifi.flow.configuration.file
+  static constexpr const char *nifi_version = "nifi.version";
   static constexpr const char *nifi_default_directory = "nifi.default.directory";
   static constexpr const char *nifi_flow_configuration_file = "nifi.flow.configuration.file";
   static constexpr const char *nifi_flow_configuration_encrypt = "nifi.flow.configuration.encrypt";
@@ -45,13 +53,18 @@ class Configuration : public Properties {
   static constexpr const char *nifi_bored_yield_duration = "nifi.bored.yield.duration";
   static constexpr const char *nifi_graceful_shutdown_seconds = "nifi.flowcontroller.graceful.shutdown.period";
   static constexpr const char *nifi_flowcontroller_drain_timeout = "nifi.flowcontroller.drain.timeout";
-  static constexpr const char *nifi_log_level = "nifi.log.level";
   static constexpr const char *nifi_server_name = "nifi.server.name";
   static constexpr const char *nifi_configuration_class_name = "nifi.flow.configuration.class.name";
   static constexpr const char *nifi_flow_repository_class_name = "nifi.flowfile.repository.class.name";
   static constexpr const char *nifi_content_repository_class_name = "nifi.content.repository.class.name";
-  static constexpr const char *nifi_volatile_repository_options = "nifi.volatile.repository.options.";
   static constexpr const char *nifi_provenance_repository_class_name = "nifi.provenance.repository.class.name";
+  static constexpr const char *nifi_volatile_repository_options_flowfile_max_count = "nifi.volatile.repository.options.flowfile.max.count";
+  static constexpr const char *nifi_volatile_repository_options_flowfile_max_bytes = "nifi.volatile.repository.options.flowfile.max.bytes";
+  static constexpr const char *nifi_volatile_repository_options_provenance_max_count = "nifi.volatile.repository.options.provenance.max.count";
+  static constexpr const char *nifi_volatile_repository_options_provenance_max_bytes = "nifi.volatile.repository.options.provenance.max.bytes";
+  static constexpr const char *nifi_volatile_repository_options_content_max_count = "nifi.volatile.repository.options.content.max.count";
+  static constexpr const char *nifi_volatile_repository_options_content_max_bytes = "nifi.volatile.repository.options.content.max.bytes";
+  static constexpr const char *nifi_volatile_repository_options_content_minimal_locking = "nifi.volatile.repository.options.content.minimal.locking";
   static constexpr const char *nifi_server_port = "nifi.server.port";
   static constexpr const char *nifi_server_report_interval = "nifi.server.report.interval";
   static constexpr const char *nifi_provenance_repository_max_storage_size = "nifi.provenance.repository.max.storage.size";
@@ -63,7 +76,12 @@ class Configuration : public Properties {
   static constexpr const char *nifi_dbcontent_repository_directory_default = "nifi.database.content.repository.directory.default";
   static constexpr const char *nifi_remote_input_secure = "nifi.remote.input.secure";
   static constexpr const char *nifi_remote_input_http = "nifi.remote.input.http.enabled";
+  static constexpr const char *nifi_remote_input_socket_port = "nifi.remote.input.socket.port";
   static constexpr const char *nifi_security_need_ClientAuth = "nifi.security.need.ClientAuth";
+  static constexpr const char *nifi_sensitive_props_additional_keys = "nifi.sensitive.props.additional.keys";
+  static constexpr const char *nifi_python_processor_dir = "nifi.python.processor.dir";
+  static constexpr const char *nifi_extension_path = "nifi.extension.path";
+
   // site2site security config
   static constexpr const char *nifi_security_client_certificate = "nifi.security.client.certificate";
   static constexpr const char *nifi_security_client_private_key = "nifi.security.client.private.key";
@@ -79,6 +97,7 @@ class Configuration : public Properties {
   // nifi rest api user name and password
   static constexpr const char *nifi_rest_api_user_name = "nifi.rest.api.user.name";
   static constexpr const char *nifi_rest_api_password = "nifi.rest.api.password";
+
   // c2 options
   static constexpr const char *nifi_c2_enable = "nifi.c2.enable";
   static constexpr const char *nifi_c2_file_watch = "nifi.c2.file.watch";
@@ -86,6 +105,27 @@ class Configuration : public Properties {
   static constexpr const char *nifi_c2_flow_url = "nifi.c2.flow.url";
   static constexpr const char *nifi_c2_flow_base_url = "nifi.c2.flow.base.url";
   static constexpr const char *nifi_c2_full_heartbeat = "nifi.c2.full.heartbeat";
+  static constexpr const char *nifi_c2_coap_connector_service = "nifi.c2.coap.connector.service";
+  static constexpr const char *nifi_c2_agent_heartbeat_period = "nifi.c2.agent.heartbeat.period";
+  static constexpr const char *nifi_c2_agent_class = "nifi.c2.agent.class";
+  static constexpr const char *nifi_c2_agent_heartbeat_reporter_classes = "nifi.c2.agent.heartbeat.reporter.classes";
+  static constexpr const char *nifi_c2_agent_coap_host = "nifi.c2.agent.coap.host";
+  static constexpr const char *nifi_c2_agent_coap_port = "nifi.c2.agent.coap.port";
+  static constexpr const char *nifi_c2_agent_protocol_class = "nifi.c2.agent.protocol.class";
+  static constexpr const char *nifi_c2_agent_identifier = "nifi.c2.agent.identifier";
+  static constexpr const char *nifi_c2_agent_trigger_classes = "nifi.c2.agent.trigger.classes";
+  static constexpr const char *nifi_c2_root_classes = "nifi.c2.root.classes";
+  static constexpr const char *nifi_c2_root_class_definitions = "nifi.c2.root.class.definitions";
+  static constexpr const char *nifi_c2_rest_listener_port = "nifi.c2.rest.listener.port";
+  static constexpr const char *nifi_c2_rest_listener_cacert = "nifi.c2.rest.listener.cacert";
+  static constexpr const char *nifi_c2_rest_url = "nifi.c2.rest.url";
+  static constexpr const char *nifi_c2_rest_url_ack = "nifi.c2.rest.url.ack";
+  static constexpr const char *nifi_c2_rest_ssl_context_service = "nifi.c2.rest.ssl.context.service";
+  static constexpr const char *nifi_c2_rest_listener_heartbeat_rooturi = "nifi.c2.rest.listener.heartbeat.rooturi";
+  static constexpr const char *nifi_c2_rest_heartbeat_minimize_updates = "nifi.c2.rest.heartbeat.minimize.updates";
+  static constexpr const char *nifi_c2_mqtt_connector_service = "nifi.c2.mqtt.connector.service";
+  static constexpr const char *nifi_c2_mqtt_heartbeat_topic = "nifi.c2.mqtt.heartbeat.topic";
+  static constexpr const char *nifi_c2_mqtt_update_topic = "nifi.c2.mqtt.update.topic";
 
   // state management options
   static constexpr const char *nifi_state_management_provider_local = "nifi.state.management.provider.local";
@@ -99,6 +139,30 @@ class Configuration : public Properties {
   static constexpr const char *minifi_disk_space_watchdog_interval = "minifi.disk.space.watchdog.interval";
   static constexpr const char *minifi_disk_space_watchdog_stop_threshold = "minifi.disk.space.watchdog.stop.threshold";
   static constexpr const char *minifi_disk_space_watchdog_restart_threshold = "minifi.disk.space.watchdog.restart.threshold";
+
+  // JNI options
+  static constexpr const char *nifi_framework_dir = "nifi.framework.dir";
+  static constexpr const char *nifi_jvm_options = "nifi.jvm.options";
+  static constexpr const char *nifi_nar_directory = "nifi.nar.directory";
+  static constexpr const char *nifi_nar_deploy_directory = "nifi.nar.deploy.directory";
+
+  // Log options
+  static constexpr const char *nifi_log_spdlog_pattern = "nifi.log.spdlog.pattern";
+  static constexpr const char *nifi_log_spdlog_shorten_names = "nifi.log.spdlog.shorten_names";
+  static constexpr const char *nifi_log_appender_rolling = "nifi.log.appender.rolling";
+  static constexpr const char *nifi_log_appender_rolling_directory = "nifi.log.appender.rolling.directory";
+  static constexpr const char *nifi_log_appender_rolling_file_name = "nifi.log.appender.rolling.file_name";
+  static constexpr const char *nifi_log_appender_rolling_max_files = "nifi.log.appender.rolling.max_files";
+  static constexpr const char *nifi_log_appender_rolling_max_file_size = "nifi.log.appender.rolling.max_file_size";
+  static constexpr const char *nifi_log_appender_stdout = "nifi.log.appender.stdout";
+  static constexpr const char *nifi_log_appender_stderr = "nifi.log.appender.stderr";
+  static constexpr const char *nifi_log_appender_null = "nifi.log.appender.null";
+  static constexpr const char *nifi_log_appender_syslog = "nifi.log.appender.syslog";
+  static constexpr const char *nifi_log_logger_root = "nifi.log.logger.root";
+  static constexpr const char *nifi_log_compression_cached_log_max_size = "nifi.log.compression.cached.log.max.size";
+  static constexpr const char *nifi_log_compression_compressed_log_max_size = "nifi.log.compression.compressed.log.max.size";
+
+  MINIFIAPI static const std::vector<core::ConfigurationProperty> CONFIGURATION_PROPERTIES;
 };
 
 }  // namespace minifi
diff --git a/libminifi/src/Configuration.cpp b/libminifi/src/Configuration.cpp
index 6a7f92a..3e916ec 100644
--- a/libminifi/src/Configuration.cpp
+++ b/libminifi/src/Configuration.cpp
@@ -15,69 +15,118 @@
  * limitations under the License.
  */
 #include "properties/Configuration.h"
+#include "core/Property.h"
 
-namespace org {
-namespace apache {
-namespace nifi {
-namespace minifi {
+namespace org::apache::nifi::minifi {
 
-constexpr const char *Configuration::nifi_default_directory;
-constexpr const char *Configuration::nifi_c2_enable;
-constexpr const char *Configuration::nifi_flow_configuration_file;
-constexpr const char *Configuration::nifi_flow_configuration_file_exit_failure;
-constexpr const char *Configuration::nifi_flow_configuration_file_backup_update;
-constexpr const char *Configuration::nifi_flow_engine_threads;
-constexpr const char *Configuration::nifi_flow_engine_alert_period;
-constexpr const char *Configuration::nifi_flow_engine_event_driven_time_slice;
-constexpr const char *Configuration::nifi_administrative_yield_duration;
-constexpr const char *Configuration::nifi_bored_yield_duration;
-constexpr const char *Configuration::nifi_graceful_shutdown_seconds;
-constexpr const char *Configuration::nifi_flowcontroller_drain_timeout;
-constexpr const char *Configuration::nifi_log_level;
-constexpr const char *Configuration::nifi_server_name;
-constexpr const char *Configuration::nifi_configuration_class_name;
-constexpr const char *Configuration::nifi_flow_repository_class_name;
-constexpr const char *Configuration::nifi_content_repository_class_name;
-constexpr const char *Configuration::nifi_volatile_repository_options;
-constexpr const char *Configuration::nifi_provenance_repository_class_name;
-constexpr const char *Configuration::nifi_server_port;
-constexpr const char *Configuration::nifi_server_report_interval;
-constexpr const char *Configuration::nifi_provenance_repository_max_storage_size;
-constexpr const char *Configuration::nifi_provenance_repository_max_storage_time;
-constexpr const char *Configuration::nifi_provenance_repository_directory_default;
-constexpr const char *Configuration::nifi_flowfile_repository_max_storage_size;
-constexpr const char *Configuration::nifi_flowfile_repository_max_storage_time;
-constexpr const char *Configuration::nifi_flowfile_repository_directory_default;
-constexpr const char *Configuration::nifi_dbcontent_repository_directory_default;
-constexpr const char *Configuration::nifi_remote_input_secure;
-constexpr const char *Configuration::nifi_remote_input_http;
-constexpr const char *Configuration::nifi_security_need_ClientAuth;
-constexpr const char *Configuration::nifi_security_client_certificate;
-constexpr const char *Configuration::nifi_security_client_private_key;
-constexpr const char *Configuration::nifi_security_client_pass_phrase;
-constexpr const char *Configuration::nifi_security_client_ca_certificate;
-constexpr const char *Configuration::nifi_security_use_system_cert_store;
-constexpr const char *Configuration::nifi_security_windows_cert_store_location;
-constexpr const char *Configuration::nifi_security_windows_server_cert_store;
-constexpr const char *Configuration::nifi_security_windows_client_cert_store;
-constexpr const char *Configuration::nifi_security_windows_client_cert_cn;
-constexpr const char *Configuration::nifi_security_windows_client_cert_key_usage;
-constexpr const char *Configuration::nifi_rest_api_user_name;
-constexpr const char *Configuration::nifi_rest_api_password;
-constexpr const char *Configuration::nifi_c2_file_watch;
-constexpr const char *Configuration::nifi_c2_flow_id;
-constexpr const char *Configuration::nifi_c2_flow_url;
-constexpr const char *Configuration::nifi_c2_flow_base_url;
-constexpr const char *Configuration::nifi_c2_full_heartbeat;
-constexpr const char *Configuration::nifi_state_management_provider_local;
-constexpr const char *Configuration::nifi_state_management_provider_local_always_persist;
-constexpr const char *Configuration::nifi_state_management_provider_local_auto_persistence_interval;
-constexpr const char *Configuration::minifi_disk_space_watchdog_enable;
-constexpr const char *Configuration::minifi_disk_space_watchdog_interval;
-constexpr const char *Configuration::minifi_disk_space_watchdog_stop_threshold;
-constexpr const char *Configuration::minifi_disk_space_watchdog_restart_threshold;
+const std::vector<core::ConfigurationProperty> Configuration::CONFIGURATION_PROPERTIES{
+  core::ConfigurationProperty{Configuration::nifi_version},
+  core::ConfigurationProperty{Configuration::nifi_default_directory},
+  core::ConfigurationProperty{Configuration::nifi_flow_configuration_file},
+  core::ConfigurationProperty{Configuration::nifi_flow_configuration_encrypt, gsl::make_not_null(core::StandardValidators::get().BOOLEAN_VALIDATOR.get())},
+  core::ConfigurationProperty{Configuration::nifi_flow_configuration_file_exit_failure, gsl::make_not_null(core::StandardValidators::get().BOOLEAN_VALIDATOR.get())},
+  core::ConfigurationProperty{Configuration::nifi_flow_configuration_file_backup_update, gsl::make_not_null(core::StandardValidators::get().BOOLEAN_VALIDATOR.get())},
+  core::ConfigurationProperty{Configuration::nifi_flow_engine_threads, gsl::make_not_null(core::StandardValidators::get().UNSIGNED_INT_VALIDATOR.get())},
+  core::ConfigurationProperty{Configuration::nifi_flow_engine_alert_period, gsl::make_not_null(core::StandardValidators::get().UNSIGNED_INT_VALIDATOR.get())},
+  core::ConfigurationProperty{Configuration::nifi_flow_engine_event_driven_time_slice, gsl::make_not_null(core::StandardValidators::get().UNSIGNED_INT_VALIDATOR.get())},
+  core::ConfigurationProperty{Configuration::nifi_administrative_yield_duration, gsl::make_not_null(core::StandardValidators::get().TIME_PERIOD_VALIDATOR.get())},
+  core::ConfigurationProperty{Configuration::nifi_bored_yield_duration, gsl::make_not_null(core::StandardValidators::get().TIME_PERIOD_VALIDATOR.get())},
+  core::ConfigurationProperty{Configuration::nifi_graceful_shutdown_seconds, gsl::make_not_null(core::StandardValidators::get().UNSIGNED_INT_VALIDATOR.get())},
+  core::ConfigurationProperty{Configuration::nifi_flowcontroller_drain_timeout, gsl::make_not_null(core::StandardValidators::get().TIME_PERIOD_VALIDATOR.get())},
+  core::ConfigurationProperty{Configuration::nifi_server_name},
+  core::ConfigurationProperty{Configuration::nifi_configuration_class_name},
+  core::ConfigurationProperty{Configuration::nifi_flow_repository_class_name},
+  core::ConfigurationProperty{Configuration::nifi_content_repository_class_name},
+  core::ConfigurationProperty{Configuration::nifi_provenance_repository_class_name},
+  core::ConfigurationProperty{Configuration::nifi_volatile_repository_options_flowfile_max_count, gsl::make_not_null(core::StandardValidators::get().UNSIGNED_INT_VALIDATOR.get())},
+  core::ConfigurationProperty{Configuration::nifi_volatile_repository_options_flowfile_max_bytes, gsl::make_not_null(core::StandardValidators::get().DATA_SIZE_VALIDATOR.get())},
+  core::ConfigurationProperty{Configuration::nifi_volatile_repository_options_provenance_max_count, gsl::make_not_null(core::StandardValidators::get().UNSIGNED_INT_VALIDATOR.get())},
+  core::ConfigurationProperty{Configuration::nifi_volatile_repository_options_provenance_max_bytes, gsl::make_not_null(core::StandardValidators::get().DATA_SIZE_VALIDATOR.get())},
+  core::ConfigurationProperty{Configuration::nifi_volatile_repository_options_content_max_count, gsl::make_not_null(core::StandardValidators::get().UNSIGNED_INT_VALIDATOR.get())},
+  core::ConfigurationProperty{Configuration::nifi_volatile_repository_options_content_max_bytes, gsl::make_not_null(core::StandardValidators::get().DATA_SIZE_VALIDATOR.get())},
+  core::ConfigurationProperty{Configuration::nifi_volatile_repository_options_content_minimal_locking, gsl::make_not_null(core::StandardValidators::get().BOOLEAN_VALIDATOR.get())},
+  core::ConfigurationProperty{Configuration::nifi_server_port, gsl::make_not_null(core::StandardValidators::get().PORT_VALIDATOR.get())},
+  core::ConfigurationProperty{Configuration::nifi_server_report_interval, gsl::make_not_null(core::StandardValidators::get().TIME_PERIOD_VALIDATOR.get())},
+  core::ConfigurationProperty{Configuration::nifi_provenance_repository_max_storage_size, gsl::make_not_null(core::StandardValidators::get().DATA_SIZE_VALIDATOR.get())},
+  core::ConfigurationProperty{Configuration::nifi_provenance_repository_max_storage_time, gsl::make_not_null(core::StandardValidators::get().TIME_PERIOD_VALIDATOR.get())},
+  core::ConfigurationProperty{Configuration::nifi_provenance_repository_directory_default},
+  core::ConfigurationProperty{Configuration::nifi_flowfile_repository_max_storage_size, gsl::make_not_null(core::StandardValidators::get().DATA_SIZE_VALIDATOR.get())},
+  core::ConfigurationProperty{Configuration::nifi_flowfile_repository_max_storage_time, gsl::make_not_null(core::StandardValidators::get().TIME_PERIOD_VALIDATOR.get())},
+  core::ConfigurationProperty{Configuration::nifi_flowfile_repository_directory_default},
+  core::ConfigurationProperty{Configuration::nifi_dbcontent_repository_directory_default},
+  core::ConfigurationProperty{Configuration::nifi_remote_input_secure, gsl::make_not_null(core::StandardValidators::get().BOOLEAN_VALIDATOR.get())},
+  core::ConfigurationProperty{Configuration::nifi_remote_input_http, gsl::make_not_null(core::StandardValidators::get().BOOLEAN_VALIDATOR.get())},
+  core::ConfigurationProperty{Configuration::nifi_remote_input_socket_port, gsl::make_not_null(core::StandardValidators::get().PORT_VALIDATOR.get())},
+  core::ConfigurationProperty{Configuration::nifi_security_need_ClientAuth, gsl::make_not_null(core::StandardValidators::get().BOOLEAN_VALIDATOR.get())},
+  core::ConfigurationProperty{Configuration::nifi_sensitive_props_additional_keys},
+  core::ConfigurationProperty{Configuration::nifi_python_processor_dir},
+  core::ConfigurationProperty{Configuration::nifi_extension_path},
+  core::ConfigurationProperty{Configuration::nifi_security_client_certificate},
+  core::ConfigurationProperty{Configuration::nifi_security_client_private_key},
+  core::ConfigurationProperty{Configuration::nifi_security_client_pass_phrase},
+  core::ConfigurationProperty{Configuration::nifi_security_client_ca_certificate},
+  core::ConfigurationProperty{Configuration::nifi_security_use_system_cert_store},
+  core::ConfigurationProperty{Configuration::nifi_security_windows_cert_store_location},
+  core::ConfigurationProperty{Configuration::nifi_security_windows_server_cert_store},
+  core::ConfigurationProperty{Configuration::nifi_security_windows_client_cert_store},
+  core::ConfigurationProperty{Configuration::nifi_security_windows_client_cert_cn},
+  core::ConfigurationProperty{Configuration::nifi_security_windows_client_cert_key_usage},
+  core::ConfigurationProperty{Configuration::nifi_rest_api_user_name},
+  core::ConfigurationProperty{Configuration::nifi_rest_api_password},
+  core::ConfigurationProperty{Configuration::nifi_c2_enable, gsl::make_not_null(core::StandardValidators::get().BOOLEAN_VALIDATOR.get())},
+  core::ConfigurationProperty{Configuration::nifi_c2_file_watch},
+  core::ConfigurationProperty{Configuration::nifi_c2_flow_id},
+  core::ConfigurationProperty{Configuration::nifi_c2_flow_url},
+  core::ConfigurationProperty{Configuration::nifi_c2_flow_base_url},
+  core::ConfigurationProperty{Configuration::nifi_c2_full_heartbeat, gsl::make_not_null(core::StandardValidators::get().BOOLEAN_VALIDATOR.get())},
+  core::ConfigurationProperty{Configuration::nifi_c2_coap_connector_service},
+  core::ConfigurationProperty{Configuration::nifi_c2_agent_heartbeat_period, gsl::make_not_null(core::StandardValidators::get().UNSIGNED_INT_VALIDATOR.get())},
+  core::ConfigurationProperty{Configuration::nifi_c2_agent_heartbeat_reporter_classes},
+  core::ConfigurationProperty{Configuration::nifi_c2_agent_class},
+  core::ConfigurationProperty{Configuration::nifi_c2_agent_coap_host},
+  core::ConfigurationProperty{Configuration::nifi_c2_agent_coap_port, gsl::make_not_null(core::StandardValidators::get().PORT_VALIDATOR.get())},
+  core::ConfigurationProperty{Configuration::nifi_c2_agent_protocol_class},
+  core::ConfigurationProperty{Configuration::nifi_c2_agent_identifier},
+  core::ConfigurationProperty{Configuration::nifi_c2_agent_trigger_classes},
+  core::ConfigurationProperty{Configuration::nifi_c2_root_classes},
+  core::ConfigurationProperty{Configuration::nifi_c2_root_class_definitions},
+  core::ConfigurationProperty{Configuration::nifi_c2_rest_listener_port, gsl::make_not_null(core::StandardValidators::get().LISTEN_PORT_VALIDATOR.get())},
+  core::ConfigurationProperty{Configuration::nifi_c2_rest_listener_cacert},
+  core::ConfigurationProperty{Configuration::nifi_c2_rest_url},
+  core::ConfigurationProperty{Configuration::nifi_c2_rest_url_ack},
+  core::ConfigurationProperty{Configuration::nifi_c2_rest_ssl_context_service},
+  core::ConfigurationProperty{Configuration::nifi_c2_rest_listener_heartbeat_rooturi},
+  core::ConfigurationProperty{Configuration::nifi_c2_rest_heartbeat_minimize_updates, gsl::make_not_null(core::StandardValidators::get().BOOLEAN_VALIDATOR.get())},
+  core::ConfigurationProperty{Configuration::nifi_c2_mqtt_connector_service},
+  core::ConfigurationProperty{Configuration::nifi_c2_mqtt_heartbeat_topic},
+  core::ConfigurationProperty{Configuration::nifi_c2_mqtt_update_topic},
+  core::ConfigurationProperty{Configuration::nifi_state_management_provider_local},
+  core::ConfigurationProperty{Configuration::nifi_state_management_provider_local_class_name},
+  core::ConfigurationProperty{Configuration::nifi_state_management_provider_local_always_persist, gsl::make_not_null(core::StandardValidators::get().BOOLEAN_VALIDATOR.get())},
+  core::ConfigurationProperty{Configuration::nifi_state_management_provider_local_auto_persistence_interval, gsl::make_not_null(core::StandardValidators::get().TIME_PERIOD_VALIDATOR.get())},
+  core::ConfigurationProperty{Configuration::nifi_state_management_provider_local_path},
+  core::ConfigurationProperty{Configuration::minifi_disk_space_watchdog_enable, gsl::make_not_null(core::StandardValidators::get().BOOLEAN_VALIDATOR.get())},
+  core::ConfigurationProperty{Configuration::minifi_disk_space_watchdog_interval, gsl::make_not_null(core::StandardValidators::get().TIME_PERIOD_VALIDATOR.get())},
+  core::ConfigurationProperty{Configuration::minifi_disk_space_watchdog_stop_threshold, gsl::make_not_null(core::StandardValidators::get().UNSIGNED_LONG_VALIDATOR.get())},
+  core::ConfigurationProperty{Configuration::minifi_disk_space_watchdog_restart_threshold, gsl::make_not_null(core::StandardValidators::get().UNSIGNED_LONG_VALIDATOR.get())},
+  core::ConfigurationProperty{Configuration::nifi_framework_dir},
+  core::ConfigurationProperty{Configuration::nifi_jvm_options},
+  core::ConfigurationProperty{Configuration::nifi_nar_directory},
+  core::ConfigurationProperty{Configuration::nifi_nar_deploy_directory},
+  core::ConfigurationProperty{Configuration::nifi_log_spdlog_pattern},
+  core::ConfigurationProperty{Configuration::nifi_log_spdlog_shorten_names, gsl::make_not_null(core::StandardValidators::get().BOOLEAN_VALIDATOR.get())},
+  core::ConfigurationProperty{Configuration::nifi_log_appender_rolling},
+  core::ConfigurationProperty{Configuration::nifi_log_appender_rolling_directory},
+  core::ConfigurationProperty{Configuration::nifi_log_appender_rolling_file_name},
+  core::ConfigurationProperty{Configuration::nifi_log_appender_rolling_max_files, gsl::make_not_null(core::StandardValidators::get().UNSIGNED_INT_VALIDATOR.get())},
+  core::ConfigurationProperty{Configuration::nifi_log_appender_rolling_max_file_size, gsl::make_not_null(core::StandardValidators::get().DATA_SIZE_VALIDATOR.get())},
+  core::ConfigurationProperty{Configuration::nifi_log_appender_stdout},
+  core::ConfigurationProperty{Configuration::nifi_log_appender_stderr},
+  core::ConfigurationProperty{Configuration::nifi_log_appender_null},
+  core::ConfigurationProperty{Configuration::nifi_log_appender_syslog},
+  core::ConfigurationProperty{Configuration::nifi_log_logger_root},
+  core::ConfigurationProperty{Configuration::nifi_log_compression_cached_log_max_size, gsl::make_not_null(core::StandardValidators::get().DATA_SIZE_VALIDATOR.get())},
+  core::ConfigurationProperty{Configuration::nifi_log_compression_compressed_log_max_size, gsl::make_not_null(core::StandardValidators::get().DATA_SIZE_VALIDATOR.get())}
+};
 
-} /* namespace minifi */
-} /* namespace nifi */
-} /* namespace apache */
-} /* namespace org */
+}  // namespace org::apache::nifi::minifi
diff --git a/libminifi/src/Configure.cpp b/libminifi/src/Configure.cpp
index 62605b8..55b66f0 100644
--- a/libminifi/src/Configure.cpp
+++ b/libminifi/src/Configure.cpp
@@ -22,6 +22,7 @@
 
 #include "core/logging/LoggerConfiguration.h"
 #include "utils/StringUtils.h"
+#include "properties/Configuration.h"
 
 namespace org {
 namespace apache {
@@ -69,7 +70,7 @@ bool Configure::isEncrypted(const std::string& key) const {
 
 std::optional<std::string> Configure::getAgentClass() const {
   std::string agent_class;
-  if (get("nifi.c2.agent.class", "c2.agent.class", agent_class) && !agent_class.empty()) {
+  if (get(Configuration::nifi_c2_agent_class, "c2.agent.class", agent_class) && !agent_class.empty()) {
     return agent_class;
   }
   return {};
@@ -77,7 +78,7 @@ std::optional<std::string> Configure::getAgentClass() const {
 
 std::string Configure::getAgentIdentifier() const {
   std::string agent_id;
-  if (!get("nifi.c2.agent.identifier", "c2.agent.identifier", agent_id) || agent_id.empty()) {
+  if (!get(Configuration::nifi_c2_agent_identifier, "c2.agent.identifier", agent_id) || agent_id.empty()) {
     std::lock_guard<std::mutex> guard(fallback_identifier_mutex_);
     return fallback_identifier_;
   }
diff --git a/libminifi/src/FlowController.cpp b/libminifi/src/FlowController.cpp
index 77a61aa..0962cec 100644
--- a/libminifi/src/FlowController.cpp
+++ b/libminifi/src/FlowController.cpp
@@ -432,6 +432,7 @@ int16_t FlowController::clearConnection(const std::string &connection) {
 
 std::shared_ptr<state::response::ResponseNode> FlowController::getAgentManifest() const {
   auto agentInfo = std::make_shared<state::response::AgentInformation>("agentInfo");
+  agentInfo->setUpdatePolicyController(std::static_pointer_cast<controllers::UpdatePolicyControllerService>(getControllerService(c2::C2Agent::UPDATE_NAME)).get());
   agentInfo->setAgentIdentificationProvider(configuration_);
   agentInfo->includeAgentStatus(false);
   return agentInfo;
diff --git a/libminifi/src/c2/C2Agent.cpp b/libminifi/src/c2/C2Agent.cpp
index e79449a..a45379a 100644
--- a/libminifi/src/c2/C2Agent.cpp
+++ b/libminifi/src/c2/C2Agent.cpp
@@ -138,7 +138,7 @@ void C2Agent::configure(const std::shared_ptr<Configure> &configure, bool reconf
   std::string clazz, heartbeat_period, device;
 
   if (!reconfigure) {
-    if (!configure->get("nifi.c2.agent.protocol.class", "c2.agent.protocol.class", clazz)) {
+    if (!configure->get(Configuration::nifi_c2_agent_protocol_class, "c2.agent.protocol.class", clazz)) {
       clazz = "RESTSender";
     }
     logger_->log_info("Class is %s", clazz);
@@ -150,7 +150,7 @@ void C2Agent::configure(const std::shared_ptr<Configure> &configure, bool reconf
       if (!protocol) {
         const char* errmsg = "Attempted to load RESTSender. To enable C2, please specify an active protocol for this agent.";
         logger_->log_error(errmsg);
-        throw minifi::Exception{ minifi::GENERAL_EXCEPTION, errmsg };
+        throw Exception{ GENERAL_EXCEPTION, errmsg };
       }
 
       logger_->log_info("Class is RESTSender");
@@ -164,7 +164,7 @@ void C2Agent::configure(const std::shared_ptr<Configure> &configure, bool reconf
     protocol_.load()->update(configure);
   }
 
-  if (configure->get("nifi.c2.agent.heartbeat.period", "c2.agent.heartbeat.period", heartbeat_period)) {
+  if (configure->get(Configuration::nifi_c2_agent_heartbeat_period, "c2.agent.heartbeat.period", heartbeat_period)) {
     try {
       if (auto heartbeat_period_ms = utils::timeutils::StringToDuration<std::chrono::milliseconds>(heartbeat_period)) {
         heart_beat_period_ = *heartbeat_period_ms;
@@ -181,7 +181,7 @@ void C2Agent::configure(const std::shared_ptr<Configure> &configure, bool reconf
   }
 
   std::string heartbeat_reporters;
-  if (configure->get("nifi.c2.agent.heartbeat.reporter.classes", "c2.agent.heartbeat.reporter.classes", heartbeat_reporters)) {
+  if (configure->get(Configuration::nifi_c2_agent_heartbeat_reporter_classes, "c2.agent.heartbeat.reporter.classes", heartbeat_reporters)) {
     std::vector<std::string> reporters = utils::StringUtils::splitAndTrim(heartbeat_reporters, ",");
     std::lock_guard<std::mutex> lock(heartbeat_mutex);
     for (const auto& reporter : reporters) {
@@ -196,7 +196,7 @@ void C2Agent::configure(const std::shared_ptr<Configure> &configure, bool reconf
   }
 
   std::string trigger_classes;
-  if (configure->get("nifi.c2.agent.trigger.classes", "c2.agent.trigger.classes", trigger_classes)) {
+  if (configure->get(Configuration::nifi_c2_agent_trigger_classes, "c2.agent.trigger.classes", trigger_classes)) {
     std::vector<std::string> triggers = utils::StringUtils::splitAndTrim(trigger_classes, ",");
     std::lock_guard<std::mutex> lock(heartbeat_mutex);
     for (const auto& trigger : triggers) {
@@ -255,10 +255,10 @@ void C2Agent::performHeartBeat() {
 }
 
 void C2Agent::serializeMetrics(C2Payload &metric_payload, const std::string &name, const std::vector<state::response::SerializedResponseNode> &metrics, bool is_container, bool is_collapsible) {
-  const auto payloads = std::count_if(begin(metrics), end(metrics), [](const state::response::SerializedResponseNode& metric) { return !metric.children.empty(); });
+  const auto payloads = std::count_if(begin(metrics), end(metrics), [](const state::response::SerializedResponseNode& metric) { return !metric.children.empty() || metric.keep_empty; });
   metric_payload.reservePayloads(metric_payload.getNestedPayloads().size() + payloads);
   for (const auto &metric : metrics) {
-    if (metric.children.size() > 0) {
+    if (metric.children.size() > 0 || (metric.children.size() == 0 && metric.keep_empty)) {
       C2Payload child_metric_payload(metric_payload.getOperation());
       if (metric.array) {
         child_metric_payload.setContainer(true);
@@ -426,180 +426,220 @@ C2Payload C2Agent::prepareConfigurationOptions(const C2ContentResponse &resp) co
     return response;
 }
 
+void C2Agent::handle_clear(const C2ContentResponse &resp) {
+  ClearOperand operand;
+  try {
+    operand = ClearOperand::parse(resp.name.c_str());
+  } catch(const std::runtime_error&) {
+    logger_->log_debug("Clearing unknown %s", resp.name);
+    return;
+  }
+
+  switch (operand.value()) {
+    case ClearOperand::CONNECTION: {
+      for (const auto& connection : resp.operation_arguments) {
+        logger_->log_debug("Clearing connection %s", connection.second.to_string());
+        update_sink_->clearConnection(connection.second.to_string());
+      }
+      break;
+    }
+    case ClearOperand::REPOSITORIES: {
+      update_sink_->drainRepositories();
+      break;
+    }
+    case ClearOperand::CORECOMPONENTSTATE: {
+      for (const auto& corecomponent : resp.operation_arguments) {
+        std::vector<state::StateController*> components = update_sink_->getComponents(corecomponent.second.to_string());
+        auto state_manager_provider = core::ProcessContext::getStateManagerProvider(logger_, controller_, configuration_);
+        if (state_manager_provider != nullptr) {
+          for (auto* component : components) {
+            logger_->log_debug("Clearing state for component %s", component->getComponentName());
+            auto state_manager = state_manager_provider->getCoreComponentStateManager(component->getComponentUUID());
+            if (state_manager != nullptr) {
+              component->stop();
+              state_manager->clear();
+              state_manager->persist();
+              component->start();
+            } else {
+              logger_->log_warn("Failed to get StateManager for component %s", component->getComponentUUID().to_string());
+            }
+          }
+        } else {
+          logger_->log_error("Failed to get StateManagerProvider");
+        }
+      }
+      break;
+    }
+    default:
+      logger_->log_error("Unknown clear operand %s", resp.name);
+  }
+
+  C2Payload response(Operation::ACKNOWLEDGE, resp.ident, true);
+  enqueue_c2_response(std::move(response));
+}
+
 /**
  * Descriptions are special types of requests that require information
  * to be put into the acknowledgement
  */
 void C2Agent::handle_describe(const C2ContentResponse &resp) {
-  auto reporter = dynamic_cast<state::response::NodeReporter*>(update_sink_);
-  if (resp.name == "metrics") {
+  DescribeOperand operand;
+  try {
+    operand = DescribeOperand::parse(resp.name.c_str());
+  } catch(const std::runtime_error&) {
     C2Payload response(Operation::ACKNOWLEDGE, resp.ident, true);
-    if (reporter != nullptr) {
-      auto iter = resp.operation_arguments.find("metricsClass");
-      std::string metricsClass;
-      if (iter != resp.operation_arguments.end()) {
-        metricsClass = iter->second.to_string();
-      }
-      auto metricsNode = reporter->getMetricsNode(metricsClass);
-      C2Payload metrics(Operation::ACKNOWLEDGE);
-      metricsClass.empty() ? metrics.setLabel("metrics") : metrics.setLabel(metricsClass);
-      if (metricsNode) {
-        serializeMetrics(metrics, metricsNode->getName(), metricsNode->serialize(), metricsNode->isArray());
-      }
-      response.addPayload(std::move(metrics));
-    }
     enqueue_c2_response(std::move(response));
     return;
-  } else if (resp.name == "configuration") {
-    auto configOptions = prepareConfigurationOptions(resp);
-    enqueue_c2_response(std::move(configOptions));
-    return;
-  } else if (resp.name == "manifest") {
-    C2Payload response(prepareConfigurationOptions(resp));
-    if (reporter != nullptr) {
-      C2Payload agentInfo(Operation::ACKNOWLEDGE, resp.ident, true);
-      agentInfo.setLabel("agentInfo");
+  }
 
-      const auto manifest = reporter->getAgentManifest();
-      serializeMetrics(agentInfo, manifest->getName(), manifest->serialize());
-      response.addPayload(std::move(agentInfo));
-    }
-    enqueue_c2_response(std::move(response));
-    return;
-  } else if (resp.name == "jstack") {
-    if (update_sink_->isRunning()) {
-      const std::vector<BackTrace> traces = update_sink_->getTraces();
-      for (const auto &trace : traces) {
-        for (const auto & line : trace.getTraces()) {
-          logger_->log_trace("%s -- %s", trace.getName(), line);
-        }
-      }
-      auto keys = configuration_->getConfiguredKeys();
+  auto reporter = dynamic_cast<state::response::NodeReporter*>(update_sink_);
+  switch (operand.value()) {
+    case DescribeOperand::METRICS: {
       C2Payload response(Operation::ACKNOWLEDGE, resp.ident, true);
-      for (const auto &trace : traces) {
-        C2Payload options(Operation::ACKNOWLEDGE, resp.ident, true);
-        options.setLabel(trace.getName());
-        std::string value;
-        for (const auto &line : trace.getTraces()) {
-          C2ContentResponse option(Operation::ACKNOWLEDGE);
-          option.name = line;
-          option.operation_arguments[line] = line;
-          options.addContent(std::move(option));
+      if (reporter != nullptr) {
+        auto iter = resp.operation_arguments.find("metricsClass");
+        std::string metricsClass;
+        if (iter != resp.operation_arguments.end()) {
+          metricsClass = iter->second.to_string();
         }
-        response.addPayload(std::move(options));
+        auto metricsNode = reporter->getMetricsNode(metricsClass);
+        C2Payload metrics(Operation::ACKNOWLEDGE);
+        metricsClass.empty() ? metrics.setLabel("metrics") : metrics.setLabel(metricsClass);
+        if (metricsNode) {
+          serializeMetrics(metrics, metricsNode->getName(), metricsNode->serialize(), metricsNode->isArray());
+        }
+        response.addPayload(std::move(metrics));
       }
       enqueue_c2_response(std::move(response));
-      return;
+      break;
     }
-  } else if (resp.name == "corecomponentstate") {
-    C2Payload response(Operation::ACKNOWLEDGE, resp.ident, true);
-    response.setLabel("corecomponentstate");
-    C2Payload states(Operation::ACKNOWLEDGE, resp.ident, true);
-    states.setLabel("corecomponentstate");
-    auto state_manager_provider = core::ProcessContext::getStateManagerProvider(logger_, controller_, configuration_);
-    if (state_manager_provider != nullptr) {
-      auto core_component_states = state_manager_provider->getAllCoreComponentStates();
-      for (const auto& core_component_state : core_component_states) {
-        C2Payload state(Operation::ACKNOWLEDGE, resp.ident, true);
-        state.setLabel(core_component_state.first.to_string());
-        for (const auto& kv : core_component_state.second) {
-          C2ContentResponse entry(Operation::ACKNOWLEDGE);
-          entry.name = kv.first;
-          entry.operation_arguments[kv.first] = kv.second;
-          state.addContent(std::move(entry));
+    case DescribeOperand::CONFIGURATION: {
+      auto configOptions = prepareConfigurationOptions(resp);
+      enqueue_c2_response(std::move(configOptions));
+      break;
+    }
+    case DescribeOperand::MANIFEST: {
+      C2Payload response(prepareConfigurationOptions(resp));
+      if (reporter != nullptr) {
+        C2Payload agentInfo(Operation::ACKNOWLEDGE, resp.ident, true);
+        agentInfo.setLabel("agentInfo");
+
+        const auto manifest = reporter->getAgentManifest();
+        serializeMetrics(agentInfo, manifest->getName(), manifest->serialize());
+        response.addPayload(std::move(agentInfo));
+      }
+      enqueue_c2_response(std::move(response));
+      break;
+    }
+    case DescribeOperand::JSTACK: {
+      if (update_sink_->isRunning()) {
+        const std::vector<BackTrace> traces = update_sink_->getTraces();
+        for (const auto &trace : traces) {
+          for (const auto & line : trace.getTraces()) {
+            logger_->log_trace("%s -- %s", trace.getName(), line);
+          }
+        }
+        auto keys = configuration_->getConfiguredKeys();
+        C2Payload response(Operation::ACKNOWLEDGE, resp.ident, true);
+        for (const auto &trace : traces) {
+          C2Payload options(Operation::ACKNOWLEDGE, resp.ident, true);
+          options.setLabel(trace.getName());
+          std::string value;
+          for (const auto &line : trace.getTraces()) {
+            C2ContentResponse option(Operation::ACKNOWLEDGE);
+            option.name = line;
+            option.operation_arguments[line] = line;
+            options.addContent(std::move(option));
+          }
+          response.addPayload(std::move(options));
         }
-        states.addPayload(std::move(state));
+        enqueue_c2_response(std::move(response));
       }
+      break;
     }
-    response.addPayload(std::move(states));
-    enqueue_c2_response(std::move(response));
-    return;
-  }
-  C2Payload response(Operation::ACKNOWLEDGE, resp.ident, true);
-  enqueue_c2_response(std::move(response));
-}
-
-void C2Agent::handle_clear(const C2ContentResponse &resp) {
-  if (resp.name == "connection") {
-    for (const auto& connection : resp.operation_arguments) {
-      logger_->log_debug("Clearing connection %s", connection.second.to_string());
-      update_sink_->clearConnection(connection.second.to_string());
-    }
-  } else if (resp.name == "repositories") {
-    update_sink_->drainRepositories();
-  } else if (resp.name == "corecomponentstate") {
-    for (const auto& corecomponent : resp.operation_arguments) {
-      std::vector<state::StateController*> components = update_sink_->getComponents(corecomponent.second.to_string());
+    case DescribeOperand::CORECOMPONENTSTATE: {
+      C2Payload response(Operation::ACKNOWLEDGE, resp.ident, true);
+      response.setLabel("corecomponentstate");
+      C2Payload states(Operation::ACKNOWLEDGE, resp.ident, true);
+      states.setLabel("corecomponentstate");
       auto state_manager_provider = core::ProcessContext::getStateManagerProvider(logger_, controller_, configuration_);
       if (state_manager_provider != nullptr) {
-        for (auto* component : components) {
-          logger_->log_debug("Clearing state for component %s", component->getComponentName());
-          auto state_manager = state_manager_provider->getCoreComponentStateManager(component->getComponentUUID());
-          if (state_manager != nullptr) {
-            component->stop();
-            state_manager->clear();
-            state_manager->persist();
-            component->start();
-          } else {
-            logger_->log_warn("Failed to get StateManager for component %s", component->getComponentUUID().to_string());
+        auto core_component_states = state_manager_provider->getAllCoreComponentStates();
+        for (const auto& core_component_state : core_component_states) {
+          C2Payload state(Operation::ACKNOWLEDGE, resp.ident, true);
+          state.setLabel(core_component_state.first.to_string());
+          for (const auto& kv : core_component_state.second) {
+            C2ContentResponse entry(Operation::ACKNOWLEDGE);
+            entry.name = kv.first;
+            entry.operation_arguments[kv.first] = kv.second;
+            state.addContent(std::move(entry));
           }
+          states.addPayload(std::move(state));
         }
-      } else {
-        logger_->log_error("Failed to get StateManagerProvider");
       }
+      response.addPayload(std::move(states));
+      enqueue_c2_response(std::move(response));
+      break;
     }
-  } else {
-    logger_->log_error("Unknown clear operand %s", resp.name);
   }
-
-  C2Payload response(Operation::ACKNOWLEDGE, resp.ident, true);
-  enqueue_c2_response(std::move(response));
 }
 
 void C2Agent::handle_update(const C2ContentResponse &resp) {
+  UpdateOperand operand;
+  try {
+    operand = UpdateOperand::parse(resp.name.c_str());
+  } catch(const std::runtime_error&) {
+    C2Payload response(Operation::ACKNOWLEDGE, state::UpdateState::NOT_APPLIED, resp.ident, true);
+    enqueue_c2_response(std::move(response));
+    return;
+  }
+
   // we've been told to update something
-  if (resp.name == "configuration") {
-    handleConfigurationUpdate(resp);
-  } else if (resp.name == "properties") {
-    state::UpdateState result = state::UpdateState::FULLY_APPLIED;
-    for (auto entry : resp.operation_arguments) {
-      bool persist = (
-          entry.second.getAnnotation("persist")
-          | utils::map(&AnnotatedValue::to_string)
-          | utils::flatMap(utils::StringUtils::toBool)).value_or(false);
-      PropertyChangeLifetime lifetime = persist ? PropertyChangeLifetime::PERSISTENT : PropertyChangeLifetime::TRANSIENT;
-      if (!update_property(entry.first, entry.second.to_string(), lifetime)) {
+  switch (operand.value()) {
+    case UpdateOperand::CONFIGURATION: {
+      handleConfigurationUpdate(resp);
+      break;
+    }
+    case UpdateOperand::PROPERTIES: {
+      state::UpdateState result = state::UpdateState::FULLY_APPLIED;
+      for (auto entry : resp.operation_arguments) {
+        bool persist = (
+            entry.second.getAnnotation("persist")
+            | utils::map(&AnnotatedValue::to_string)
+            | utils::flatMap(utils::StringUtils::toBool)).value_or(false);
+        PropertyChangeLifetime lifetime = persist ? PropertyChangeLifetime::PERSISTENT : PropertyChangeLifetime::TRANSIENT;
+        if (!update_property(entry.first, entry.second.to_string(), lifetime)) {
+          result = state::UpdateState::PARTIALLY_APPLIED;
+        }
+      }
+      // apply changes and persist properties requested to be persisted
+      if (!configuration_->commitChanges()) {
         result = state::UpdateState::PARTIALLY_APPLIED;
       }
-    }
-    // apply changes and persist properties requested to be persisted
-    if (!configuration_->commitChanges()) {
-      result = state::UpdateState::PARTIALLY_APPLIED;
-    }
-    C2Payload response(Operation::ACKNOWLEDGE, result, resp.ident, true);
-    enqueue_c2_response(std::move(response));
-  } else if (resp.name == "c2") {
-    // prior configuration options were already in place. thus
-    // we clear the map so that we don't go through replacing
-    // unnecessary objects.
-    running_c2_configuration->clear();
-
-    for (auto entry : resp.operation_arguments) {
-      bool can_update = true;
-      if (nullptr != update_service_) {
-        can_update = update_service_->canUpdate(entry.first);
+      C2Payload response(Operation::ACKNOWLEDGE, result, resp.ident, true);
+      enqueue_c2_response(std::move(response));
+      break;
+    } case UpdateOperand::C2: {
+      // prior configuration options were already in place. thus
+      // we clear the map so that we don't go through replacing
+      // unnecessary objects.
+      running_c2_configuration->clear();
+
+      for (auto entry : resp.operation_arguments) {
+        bool can_update = true;
+        if (nullptr != update_service_) {
+          can_update = update_service_->canUpdate(entry.first);
+        }
+        if (can_update)
+          running_c2_configuration->set(entry.first, entry.second.to_string());
       }
-      if (can_update)
-        running_c2_configuration->set(entry.first, entry.second.to_string());
-    }
 
-    if (resp.operation_arguments.size() > 0)
-      configure(running_c2_configuration);
-    C2Payload response(Operation::ACKNOWLEDGE, state::UpdateState::FULLY_APPLIED, resp.ident, true);
-    enqueue_c2_response(std::move(response));
-  } else {
-    C2Payload response(Operation::ACKNOWLEDGE, state::UpdateState::NOT_APPLIED, resp.ident, true);
-    enqueue_c2_response(std::move(response));
+      if (resp.operation_arguments.size() > 0)
+        configure(running_c2_configuration);
+      C2Payload response(Operation::ACKNOWLEDGE, state::UpdateState::FULLY_APPLIED, resp.ident, true);
+      enqueue_c2_response(std::move(response));
+      break;
+    }
   }
 }
 
@@ -647,23 +687,32 @@ C2Payload C2Agent::bundleDebugInfo(std::map<std::string, std::unique_ptr<io::Inp
 }
 
 void C2Agent::handle_transfer(const C2ContentResponse &resp) {
-  if (resp.name != "debug") {
+  TransferOperand operand;
+  try {
+    operand = TransferOperand::parse(resp.name.c_str());
+  } catch(const std::runtime_error&) {
     throw C2TransferError("Unknown operand '" + resp.name + "'");
   }
-  auto target_it = resp.operation_arguments.find("target");
-  if (target_it == resp.operation_arguments.end()) {
-    throw C2DebugBundleError("Missing argument for debug operation: 'target'");
-  }
-  std::optional<std::string> url = resolveUrl(target_it->second.to_string());
-  if (!url) {
-    throw C2DebugBundleError("Invalid url");
-  }
-  std::map<std::string, std::unique_ptr<io::InputStream>> files = update_sink_->getDebugInfo();
 
-  auto bundle = bundleDebugInfo(files);
-  C2Payload &&response = protocol_.load()->consumePayload(url.value(), bundle, TRANSMIT, false);
-  if (response.getStatus().getState() == state::UpdateState::READ_ERROR) {
-    throw C2DebugBundleError("Error while uploading");
+  switch (operand.value()) {
+    case TransferOperand::DEBUG: {
+      auto target_it = resp.operation_arguments.find("target");
+      if (target_it == resp.operation_arguments.end()) {
+        throw C2DebugBundleError("Missing argument for debug operation: 'target'");
+      }
+      std::optional<std::string> url = resolveUrl(target_it->second.to_string());
+      if (!url) {
+        throw C2DebugBundleError("Invalid url");
+      }
+      std::map<std::string, std::unique_ptr<io::InputStream>> files = update_sink_->getDebugInfo();
+
+      auto bundle = bundleDebugInfo(files);
+      C2Payload &&response = protocol_.load()->consumePayload(url.value(), bundle, TRANSMIT, false);
+      if (response.getStatus().getState() == state::UpdateState::READ_ERROR) {
+        throw C2DebugBundleError("Error while uploading");
+      }
+      break;
+    }
   }
 }
 
@@ -740,14 +789,14 @@ std::optional<std::string> C2Agent::resolveFlowUrl(const std::string& url) const
     return url;
   }
   std::string base;
-  if (configuration_->get(minifi::Configure::nifi_c2_flow_base_url, base)) {
+  if (configuration_->get(Configure::nifi_c2_flow_base_url, base)) {
     base = utils::StringUtils::trim(base);
     if (!utils::StringUtils::endsWith(base, "/")) {
       base += "/";
     }
     base += url;
     return base;
-  } else if (configuration_->get("nifi.c2.rest.url", "c2.rest.url", base)) {
+  } else if (configuration_->get(Configuration::nifi_c2_rest_url, "c2.rest.url", base)) {
     utils::URL base_url{utils::StringUtils::trim(base)};
     if (base_url.isValid()) {
       return base_url.hostPort() + "/c2/api/" + url;
@@ -763,7 +812,7 @@ std::optional<std::string> C2Agent::resolveUrl(const std::string& url) const {
     return url;
   }
   std::string base;
-  if (!configuration_->get("nifi.c2.rest.url", "c2.rest.url", base)) {
+  if (!configuration_->get(Configuration::nifi_c2_rest_url, "c2.rest.url", base)) {
     logger_->log_error("Missing C2 REST URL");
     return std::nullopt;
   }
diff --git a/libminifi/src/c2/C2Client.cpp b/libminifi/src/c2/C2Client.cpp
index c9782e4..3706574 100644
--- a/libminifi/src/c2/C2Client.cpp
+++ b/libminifi/src/c2/C2Client.cpp
@@ -55,7 +55,7 @@ void C2Client::stopC2() {
 
 bool C2Client::isC2Enabled() const {
   std::string c2_enable_str;
-  configuration_->get(Configure::nifi_c2_enable, "c2.enable", c2_enable_str);
+  configuration_->get(minifi::Configuration::nifi_c2_enable, "c2.enable", c2_enable_str);
   return utils::StringUtils::toBool(c2_enable_str).value_or(false);
 }
 
@@ -85,7 +85,7 @@ void C2Client::initialize(core::controller::ControllerServiceProvider *controlle
   }
 
   std::string class_csv;
-  if (configuration_->get("nifi.c2.root.classes", class_csv)) {
+  if (configuration_->get(minifi::Configuration::nifi_c2_root_classes, class_csv)) {
     std::vector<std::string> classes = utils::StringUtils::split(class_csv, ",");
 
     for (const std::string& clazz : classes) {
@@ -105,6 +105,10 @@ void C2Client::initialize(core::controller::ControllerServiceProvider *controlle
         monitor->addRepository(flow_file_repo_);
         monitor->setStateMonitor(update_sink);
       }
+      auto agent_node = dynamic_cast<state::response::AgentNode*>(response_node.get());
+      if (agent_node != nullptr && controller != nullptr) {
+        agent_node->setUpdatePolicyController(std::static_pointer_cast<controllers::UpdatePolicyControllerService>(controller->getControllerService(C2Agent::UPDATE_NAME)).get());
+      }
       auto configuration_checksums = dynamic_cast<state::response::ConfigurationChecksums*>(response_node.get());
       if (configuration_checksums) {
         configuration_checksums->addChecksumCalculator(configuration_->getChecksumCalculator());
@@ -312,7 +316,7 @@ std::shared_ptr<state::response::ResponseNode> C2Client::getMetricsNode(const st
 
 std::vector<std::shared_ptr<state::response::ResponseNode>> C2Client::getHeartbeatNodes(bool include_manifest) const {
   std::string fullHb{"true"};
-  configuration_->get("nifi.c2.full.heartbeat", fullHb);
+  configuration_->get(minifi::Configuration::nifi_c2_full_heartbeat, fullHb);
   const bool include = include_manifest || fullHb == "true";
 
   std::vector<std::shared_ptr<state::response::ResponseNode>> nodes;
diff --git a/libminifi/src/c2/HeartbeatJsonSerializer.cpp b/libminifi/src/c2/HeartbeatJsonSerializer.cpp
index 68819c1..aa4b1e5 100644
--- a/libminifi/src/c2/HeartbeatJsonSerializer.cpp
+++ b/libminifi/src/c2/HeartbeatJsonSerializer.cpp
@@ -221,13 +221,10 @@ struct NamedValue {
       }
       return;
     }
-    rapidjson::Value& val = [&] () -> rapidjson::Value& {
-      return values[0].IsObject() && values[0].HasMember(member_key) ? values[0][member_key] : values[0];
-    }();
     if (target.IsArray()) {
-      target.PushBack(val, alloc);
+      target.PushBack(values[0], alloc);
     } else {
-      target.AddMember(member_key, val, alloc);
+      target.AddMember(member_key, values[0], alloc);
     }
   }
 };
diff --git a/libminifi/src/c2/protocols/RESTProtocol.cpp b/libminifi/src/c2/protocols/RESTProtocol.cpp
index 4776b33..2dcde52 100644
--- a/libminifi/src/c2/protocols/RESTProtocol.cpp
+++ b/libminifi/src/c2/protocols/RESTProtocol.cpp
@@ -30,6 +30,7 @@
 
 #include "core/TypedValues.h"
 #include "utils/gsl.h"
+#include "properties/Configuration.h"
 
 #undef GetObject  // windows.h #defines GetObject = GetObjectA or GetObjectW, which conflicts with rapidjson
 
@@ -143,10 +144,10 @@ RESTProtocol::RESTProtocol() = default;
 void RESTProtocol::initialize(core::controller::ControllerServiceProvider* /*controller*/, const std::shared_ptr<Configure> &configure) {
   if (configure) {
     std::string value_str;
-    if (configure->get("nifi.c2.rest.heartbeat.minimize.updates", "c2.rest.heartbeat.minimize.updates", value_str)) {
+    if (configure->get(minifi::Configuration::nifi_c2_rest_heartbeat_minimize_updates, "c2.rest.heartbeat.minimize.updates", value_str)) {
       auto opt_value = utils::StringUtils::toBool(value_str);
       if (!opt_value) {
-        logger_->log_error("Cannot convert '%s' to bool for property '%s'", value_str, "nifi.c2.rest.heartbeat.minimize.updates");
+        logger_->log_error("Cannot convert '%s' to bool for property '%s'", value_str, minifi::Configuration::nifi_c2_rest_heartbeat_minimize_updates);
         minimize_updates_ = false;
       } else {
         minimize_updates_ = opt_value.value();
diff --git a/libminifi/src/core/controller/ControllerServiceProvider.cpp b/libminifi/src/core/controller/ControllerServiceProvider.cpp
index 942c299..c871222 100644
--- a/libminifi/src/core/controller/ControllerServiceProvider.cpp
+++ b/libminifi/src/core/controller/ControllerServiceProvider.cpp
@@ -32,7 +32,7 @@ namespace controller {
  * @return the ControllerService that is registered with the given
  * identifier
  */
-std::shared_ptr<ControllerService> ControllerServiceProvider::getControllerService(const std::string &identifier) {
+std::shared_ptr<ControllerService> ControllerServiceProvider::getControllerService(const std::string &identifier) const {
   auto service = controller_map_->getControllerServiceNode(identifier);
   if (service != nullptr) {
     return service->getControllerServiceImplementation();
diff --git a/libminifi/src/core/extension/ExtensionManager.cpp b/libminifi/src/core/extension/ExtensionManager.cpp
index 2295276..8e63a80 100644
--- a/libminifi/src/core/extension/ExtensionManager.cpp
+++ b/libminifi/src/core/extension/ExtensionManager.cpp
@@ -25,6 +25,7 @@
 #include "core/extension/DynamicLibrary.h"
 #include "agent/agent_version.h"
 #include "core/extension/Utils.h"
+#include "properties/Configuration.h"
 
 namespace org {
 namespace apache {
@@ -57,7 +58,11 @@ bool ExtensionManager::initialize(const std::shared_ptr<Configure>& config) {
       return;
     }
     std::string pattern = [&] {
-      auto opt_pattern = config->get(nifi_extension_path);
+      /**
+       * Comma separated list of path patterns. Patterns prepended with "!" result in the exclusion
+       * of the extensions matching that pattern, unless some subsequent pattern re-enables it.
+       */
+      auto opt_pattern = config->get(minifi::Configuration::nifi_extension_path);
       if (!opt_pattern) {
         logger_->log_warn("No extension path is provided, using default: '%s'", DEFAULT_EXTENSION_PATH);
       }
diff --git a/libminifi/src/core/state/nodes/SupportedOperations.cpp b/libminifi/src/core/state/nodes/SupportedOperations.cpp
new file mode 100644
index 0000000..dce6625
--- /dev/null
+++ b/libminifi/src/core/state/nodes/SupportedOperations.cpp
@@ -0,0 +1,146 @@
+/**
+ *
+ * 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/nodes/SupportedOperations.h"
+#include "core/Resource.h"
+
+namespace org::apache::nifi::minifi::state::response {
+
+SupportedOperations::SupportedOperations(const std::string &name, const utils::Identifier &uuid)
+    : DeviceInformation(name, uuid) {
+  setArray(true);
+}
+
+SupportedOperations::SupportedOperations(const std::string &name)
+    : DeviceInformation(name) {
+  setArray(true);
+}
+
+std::string SupportedOperations::getName() const {
+  return "supportedOperations";
+}
+
+void SupportedOperations::addProperty(SerializedResponseNode& properties, const std::string& operand, const Metadata& metadata) {
+  SerializedResponseNode operand_node;
+  operand_node.name = operand;
+  operand_node.keep_empty = true;
+
+  for (const auto& [key, value_array] : metadata) {
+    SerializedResponseNode metadata_item;
+    metadata_item.name = key;
+    metadata_item.array = true;
+
+    for (const auto& value_object : value_array) {
+      SerializedResponseNode value_child;
+      for (const auto& pair: value_object) {
+        SerializedResponseNode object_element;
+        object_element.name = pair.first;
+        object_element.value = pair.second;
+        value_child.children.push_back(object_element);
+      }
+      metadata_item.children.push_back(value_child);
+    }
+
+    operand_node.children.push_back(metadata_item);
+  }
+
+  properties.children.push_back(operand_node);
+}
+
+SupportedOperations::Metadata SupportedOperations::buildUpdatePropertiesMetadata() const {
+  std::vector<std::unordered_map<std::string, std::string>> supported_config_updates;
+  for (const auto& config_property : minifi::Configuration::CONFIGURATION_PROPERTIES) {
+    if (!update_policy_controller_ ||
+        (update_policy_controller_ && update_policy_controller_->canUpdate(std::string(config_property.name)))) {
+      std::unordered_map<std::string, std::string> property;
+      property.emplace("propertyName", config_property.name);
+      property.emplace("validator", config_property.validator->getName());
+      supported_config_updates.push_back(property);
+    }
+  }
+  Metadata available_properties;
+  available_properties.emplace("availableProperties", supported_config_updates);
+  return available_properties;
+}
+
+void SupportedOperations::fillProperties(SerializedResponseNode& properties, minifi::c2::Operation operation) const {
+  switch (operation.value()) {
+    case minifi::c2::Operation::DESCRIBE: {
+      serializeProperty<minifi::c2::DescribeOperand>(properties);
+      break;
+    }
+    case minifi::c2::Operation::UPDATE: {
+      std::unordered_map<std::string, Metadata> operand_with_metadata;
+      operand_with_metadata.emplace("properties", buildUpdatePropertiesMetadata());
+      serializeProperty<minifi::c2::UpdateOperand>(properties, operand_with_metadata);
+      break;
+    }
+    case minifi::c2::Operation::TRANSFER: {
+      serializeProperty<minifi::c2::TransferOperand>(properties);
+      break;
+    }
+    case minifi::c2::Operation::CLEAR: {
+      serializeProperty<minifi::c2::ClearOperand>(properties);
+      break;
+    }
+    case minifi::c2::Operation::START:
+    case minifi::c2::Operation::STOP: {
+      addProperty(properties, "c2");
+      if (monitor_) {
+        for (const auto& component: monitor_->getAllComponents()) {
+          addProperty(properties, component->getComponentName());
+        }
+      }
+      break;
+    }
+    default:
+      break;
+  }
+}
+
+std::vector<SerializedResponseNode> SupportedOperations::serialize() {
+  std::vector<SerializedResponseNode> serialized;
+  SerializedResponseNode supported_operation;
+  supported_operation.name = "supportedOperations";
+  supported_operation.array = true;
+
+  for (const auto& operation : minifi::c2::Operation::values()) {
+    SerializedResponseNode child;
+    child.name = "supportedOperations";
+
+    SerializedResponseNode operation_type;
+    operation_type.name = "type";
+    operation_type.value = operation;
+
+    SerializedResponseNode properties;
+    properties.name = "properties";
+
+    fillProperties(properties, minifi::c2::Operation::parse(operation.c_str()));
+
+    child.children.push_back(operation_type);
+    child.children.push_back(properties);
+    supported_operation.children.push_back(child);
+  }
+
+  serialized.push_back(supported_operation);
+  return serialized;
+}
+
+REGISTER_RESOURCE(SupportedOperations, "Node part of an AST that defines the supported C2 operations in the Agent Manifest.");
+
+}  // namespace org::apache::nifi::minifi::state::response
diff --git a/libminifi/src/utils/ChecksumCalculator.cpp b/libminifi/src/utils/ChecksumCalculator.cpp
index 6ec5071..32afb4c 100644
--- a/libminifi/src/utils/ChecksumCalculator.cpp
+++ b/libminifi/src/utils/ChecksumCalculator.cpp
@@ -23,10 +23,11 @@
 #include "sodium/crypto_hash_sha256.h"
 #include "utils/file/FileUtils.h"
 #include "utils/StringUtils.h"
+#include "properties/Configuration.h"
 
 namespace {
 
-const std::string AGENT_IDENTIFIER_KEY = "nifi.c2.agent.identifier=";
+const std::string AGENT_IDENTIFIER_KEY = std::string(org::apache::nifi::minifi::Configuration::nifi_c2_agent_identifier) + "=";
 
 }
 
diff --git a/libminifi/test/TestBase.h b/libminifi/test/TestBase.h
index 7363775..117fb7e 100644
--- a/libminifi/test/TestBase.h
+++ b/libminifi/test/TestBase.h
@@ -37,6 +37,7 @@
 #include "core/Relationship.h"
 #include "core/repository/VolatileContentRepository.h"
 #include "utils/file/FileUtils.h"
+#include "properties/Configuration.h"
 
 namespace minifi = org::apache::nifi::minifi;
 namespace utils = minifi::utils;
@@ -373,9 +374,9 @@ static bool extensionInitializer = [] {
   LogTestController::getInstance().setTrace<core::extension::Module>();
   auto config = std::make_shared<minifi::Configure>();
 #ifdef EXTENSION_LIST
-  config->set(core::extension::nifi_extension_path, EXTENSION_LIST);
+  config->set(minifi::Configuration::nifi_extension_path, EXTENSION_LIST);
 #else
-  config->set(core::extension::nifi_extension_path, "*minifi-*");
+  config->set(minifi::Configuration::nifi_extension_path, "*minifi-*");
 #endif
   core::extension::ExtensionManager::get().initialize(config);
   return true;
diff --git a/libminifi/test/resources/TestHTTPGet.yml b/libminifi/test/resources/TestHTTPGet.yml
index 6fb9989..821e390 100644
--- a/libminifi/test/resources/TestHTTPGet.yml
+++ b/libminifi/test/resources/TestHTTPGet.yml
@@ -50,7 +50,15 @@ Processors:
       Properties:
         Log Level: info
         Log Payload: true
-
+Controller Services:
+    - id: 94491a38-015a-1000-0000-000000000001
+      name: C2UpdatePolicy
+      class: UpdatePolicyControllerService
+      Properties:
+        Allow All Properties: true
+        Disallowed Properties:
+          - value: nifi.extension.path
+          - value: nifi.python.processor.dir
 Connections:
     - name: TransferFilesToRPG
       id: 2438e3c8-015a-1000-79ca-83af40ec1997
diff --git a/libminifi/test/sensors-tests/SensorTests.cpp b/libminifi/test/sensors-tests/SensorTests.cpp
index f5a90c9..a91e760 100644
--- a/libminifi/test/sensors-tests/SensorTests.cpp
+++ b/libminifi/test/sensors-tests/SensorTests.cpp
@@ -76,7 +76,7 @@ class PcapTestHarness : public IntegrationBase {
     auto inv = dynamic_cast<minifi::processors::GetEnvironmentalSensors*>(proc);
     assert(inv != nullptr);
 
-    configuration->set("nifi.c2.enable", "false");
+    configuration->set(org::apache::nifi::minifi::Configuration::nifi_c2_enable, "false");
   }
 
  protected:
diff --git a/libminifi/test/unit/DecryptorTests.cpp b/libminifi/test/unit/DecryptorTests.cpp
index 095ca6b..558978b 100644
--- a/libminifi/test/unit/DecryptorTests.cpp
+++ b/libminifi/test/unit/DecryptorTests.cpp
@@ -101,7 +101,7 @@ TEST_CASE("Decryptor can decrypt a configuration file", "[decryptSensitiveProper
   REQUIRE(*password == "OpenSesame");
 
   std::string agent_identifier;
-  REQUIRE(configuration.get("nifi.c2.agent.identifier", "c2.agent.identifier", agent_identifier));
+  REQUIRE(configuration.get(minifi::Configuration::nifi_c2_agent_identifier, "c2.agent.identifier", agent_identifier));
   REQUIRE(agent_identifier == "TailFileTester-001");
 
   const auto unencrypted_property = configuration.get(minifi::Configure::nifi_bored_yield_duration);
diff --git a/libminifi/test/unit/SocketTests.cpp b/libminifi/test/unit/SocketTests.cpp
index 54a44e4..7e3d835 100644
--- a/libminifi/test/unit/SocketTests.cpp
+++ b/libminifi/test/unit/SocketTests.cpp
@@ -28,6 +28,7 @@
 #include "io/StreamFactory.h"
 #include "io/Sockets.h"
 #include "utils/ThreadPool.h"
+#include "properties/Configuration.h"
 
 namespace minifi = org::apache::nifi::minifi;
 namespace io = minifi::io;
@@ -206,7 +207,7 @@ TEST_CASE("TestTLSContextCreation", "[TestSocket8]") {
  */
 TEST_CASE("TestTLSContextCreation2", "[TestSocket9]") {
   std::shared_ptr<minifi::Configure> configure = std::make_shared<minifi::Configure>();
-  configure->set("nifi.remote.input.secure", "false");
+  configure->set(minifi::Configuration::nifi_remote_input_secure, "false");
   auto factory = io::StreamFactory::getInstance(configure);
   std::string host = Socket::getMyHostName();
   Socket *socket = factory->createSocket(host, 10001).release();
@@ -220,7 +221,7 @@ TEST_CASE("TestTLSContextCreation2", "[TestSocket9]") {
  */
 TEST_CASE("TestTLSContextCreationNullptr", "[TestSocket10]") {
   std::shared_ptr<minifi::Configure> configure = std::make_shared<minifi::Configure>();
-  configure->set("nifi.remote.input.secure", "false");
+  configure->set(minifi::Configuration::nifi_remote_input_secure, "false");
   auto factory = io::StreamFactory::getInstance(configure);
   std::string host = Socket::getMyHostName();
   io::Socket *socket = factory->createSecureSocket(host, 10001, nullptr).release();
diff --git a/nanofi/include/cxx/Instance.h b/nanofi/include/cxx/Instance.h
index 8f310a4..89b9a30 100644
--- a/nanofi/include/cxx/Instance.h
+++ b/nanofi/include/cxx/Instance.h
@@ -102,8 +102,8 @@ class Instance {
   void enableAsyncC2(C2_Server *server, c2_stop_callback *c1, c2_start_callback* /*c2*/, c2_update_callback* /*c3*/) {
     running_ = true;
     if (server->type != C2_Server_Type::MQTT) {
-      configure_->set("c2.rest.url", server->url);
-      configure_->set("c2.rest.url.ack", server->ack_url);
+      configure_->set(minifi::Configuration::nifi_c2_rest_url, server->url);
+      configure_->set(minifi::Configuration::nifi_c2_rest_url_ack, server->ack_url);
     }
     agent_ = std::make_shared<c2::C2CallbackAgent>(nullptr, nullptr, nullptr, configure_);
     listener_thread_pool_.start();