You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by sz...@apache.org on 2022/06/14 14:16:47 UTC

[nifi-minifi-cpp] branch main updated: MINIFICPP-1803 Create component properties statically

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

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


The following commit(s) were added to refs/heads/main by this push:
     new c438d9fe6 MINIFICPP-1803 Create component properties statically
c438d9fe6 is described below

commit c438d9fe6eb0d3f34e34ede2501f9399dfafb77f
Author: Ferenc Gerlits <fg...@gmail.com>
AuthorDate: Tue Jun 14 15:24:43 2022 +0200

    MINIFICPP-1803 Create component properties statically
    
    ... so that we don't need to instantiate every processor etc when
    creating the manifest.
    
    Processor and ControllerService properties, relationships and other
    attributes are now static, and can be read without instantiating the
    class.
    
    There are 4 cases, represented by the new ResourceType enum:
    
    - Processor: a subclass of Processor, used to be registered using
      REGISTER_RESOURCE[_AS]
    - ControllerService: a subclass of ControllerService, used to be
      registered using REGISTER_RESOURCE[_AS]
    - InternalResource: a subclass of either Processor or ControllerService,
      used to be registered using REGISTER_INTERNAL_RESOUCE[_AS]
    - DescriptionOnly: used to be registered using REGISTER_RESOURCE[_AS],
      but not a subclass of either Processor or ControllerService.
    
    Ideally, properties() and relationships() should be constexpr variables
    instead of functions, too. This will be done later as part of
    https://issues.apache.org/jira/browse/MINIFICPP-1825.
    
    The manifest has mostly stayed identical, with these differences:
    
    - minifi-coap:
      - CoapConnectorService is no longer included in the manifest. Since
        this controller service is marked as 'internal resource', I think
        this is correct.
    
    - minifi-jni:
      - ExecuteJavaProcessor was not included in the manifest, but now it
        is. I don't know why it wasn't included earlier.
      - The descriptions of ExecuteJavaControllerService and
        JavaControllerService are slightly different: I have deleted a space
        from the end.
    
    - minifi-mqtt:
      - ConvertHeartbeat, ConvertJSONAck, ConvertUpdate used to be included
        in the manifest, and they aren't any longer. Since these processors
        are marked as 'internal resource', I think this is correct.
    
    - minifi-script-extensions:
      - The ReloadOnScriptChange property was not included in the
        ExecutePythonProcessor section of the manifest. This was a bug,
        fixed now.
    
    - minifi-sensors:
      - This extension was missing from the manifest on main, not sure why.
    
    - minifi-standard-processors:
      - The description of ExecuteProcess is slightly different: I have
        added 2 missing spaces.
    
    - minifi-system:
      - The Battery Status Discharge property of LinuxPowerManagerService
        was missing from the old manifest. This was a bug, fixed now.
      - The description of NetworkPrioritizerService is slightly diferent:
        fixed typo ouput -> output.
    
    - minifi-test-processors:
      - The description of KamikazeProcessor is slightly diferent: fixed
        typo configration -> configuration.
      - Not sure if we should include this artifact in the manifest, but we
        did, so we still do.
    
    Closes #1334
    Signed-off-by: Marton Szasz <sz...@apache.org>
---
 CONTRIB.md                                         |   6 +-
 Extensions.md                                      |  12 +-
 .../controllerservices/AWSCredentialsService.cpp   |  12 +-
 .../aws/controllerservices/AWSCredentialsService.h |  23 +-
 extensions/aws/processors/DeleteS3Object.cpp       |  40 +-
 extensions/aws/processors/DeleteS3Object.h         |  48 +--
 extensions/aws/processors/FetchS3Object.cpp        |  30 +-
 extensions/aws/processors/FetchS3Object.h          |  49 ++-
 extensions/aws/processors/ListS3.cpp               |  69 +---
 extensions/aws/processors/ListS3.h                 |  61 +--
 extensions/aws/processors/PutS3Object.cpp          |  94 +----
 extensions/aws/processors/PutS3Object.h            |  61 ++-
 extensions/aws/processors/S3Processor.cpp          |  95 +----
 extensions/aws/processors/S3Processor.h            |  45 ++-
 .../processors/S3ProcessorStaticDefinitions.cpp    | 273 ++++++++++++++
 .../AzureStorageCredentialsService.cpp             |   5 +-
 .../AzureStorageCredentialsService.h               |  15 +
 .../processors/AzureBlobStorageProcessorBase.cpp   |  40 --
 .../processors/AzureBlobStorageProcessorBase.h     |  13 +-
 .../AzureBlobStorageSingleBlobProcessorBase.cpp    |   7 -
 .../AzureBlobStorageSingleBlobProcessorBase.h      |   5 +-
 .../AzureDataLakeStorageFileProcessorBase.cpp      |   6 -
 .../AzureDataLakeStorageFileProcessorBase.h        |   8 +-
 .../AzureDataLakeStorageProcessorBase.cpp          |  13 -
 .../processors/AzureDataLakeStorageProcessorBase.h |   8 +-
 .../azure/processors/AzureStorageProcessorBase.cpp |   5 -
 .../azure/processors/AzureStorageProcessorBase.h   |   2 +-
 .../AzureStorageProcessorStaticDefinitions.cpp     | 291 +++++++++++++++
 .../azure/processors/DeleteAzureBlobStorage.cpp    |  34 +-
 .../azure/processors/DeleteAzureBlobStorage.h      |  30 +-
 .../processors/DeleteAzureDataLakeStorage.cpp      |  20 +-
 .../azure/processors/DeleteAzureDataLakeStorage.h  |  25 +-
 .../azure/processors/FetchAzureBlobStorage.cpp     |  36 +-
 .../azure/processors/FetchAzureBlobStorage.h       |  25 +-
 .../azure/processors/FetchAzureDataLakeStorage.cpp |  42 +--
 .../azure/processors/FetchAzureDataLakeStorage.h   |  35 +-
 .../azure/processors/ListAzureBlobStorage.cpp      |  38 +-
 extensions/azure/processors/ListAzureBlobStorage.h |  24 +-
 .../azure/processors/ListAzureDataLakeStorage.cpp  |  43 +--
 .../azure/processors/ListAzureDataLakeStorage.h    |  25 +-
 .../azure/processors/PutAzureBlobStorage.cpp       |  34 +-
 extensions/azure/processors/PutAzureBlobStorage.h  |  22 +-
 .../azure/processors/PutAzureDataLakeStorage.cpp   |  28 +-
 .../azure/processors/PutAzureDataLakeStorage.h     |  27 +-
 extensions/bustache/ApplyTemplate.cpp              |   7 +-
 extensions/bustache/ApplyTemplate.h                |  17 +-
 extensions/civetweb/processors/ListenHTTP.cpp      |  49 +--
 extensions/civetweb/processors/ListenHTTP.h        |  67 ++--
 extensions/civetweb/protocols/RESTReceiver.cpp     |  15 +-
 extensions/civetweb/protocols/RESTReceiver.h       |  15 +-
 .../coap/controllerservice/CoapConnector.cpp       |  36 +-
 extensions/coap/controllerservice/CoapConnector.h  |  33 +-
 extensions/coap/protocols/CoapC2Protocol.cpp       |  17 +-
 extensions/coap/protocols/CoapC2Protocol.h         |  27 +-
 .../ExpressionContextBuilder.cpp                   |  16 +-
 .../expression-language/ExpressionContextBuilder.h |  23 +-
 .../tests/ProcessContextExprTests.cpp              |  25 +-
 .../tests/RouteOnAttributeTests.cpp                |   3 +-
 .../GCPCredentialsControllerService.cpp            |   5 +-
 .../GCPCredentialsControllerService.h              |  12 +
 extensions/gcp/processors/DeleteGCSObject.cpp      |  42 +--
 extensions/gcp/processors/DeleteGCSObject.h        |  27 +-
 extensions/gcp/processors/FetchGCSObject.cpp       |  43 +--
 extensions/gcp/processors/FetchGCSObject.h         |  27 +-
 extensions/gcp/processors/GCSProcessor.cpp         |  23 --
 extensions/gcp/processors/GCSProcessor.h           |   7 +
 .../processors/GCSProcessorStaticDefinitions.cpp   | 210 +++++++++++
 extensions/gcp/processors/ListGCSBucket.cpp        |  26 +-
 extensions/gcp/processors/ListGCSBucket.h          |  26 +-
 extensions/gcp/processors/PutGCSObject.cpp         |  75 +---
 extensions/gcp/processors/PutGCSObject.h           |  32 +-
 extensions/gcp/tests/DeleteGCSObjectTests.cpp      |   4 +-
 extensions/gcp/tests/FetchGCSObjectTests.cpp       |   4 +-
 .../tests/GCPCredentialsControllerServiceTests.cpp |  12 +-
 extensions/gcp/tests/ListGCSBucketTests.cpp        |   4 +-
 extensions/gcp/tests/PutGCSObjectTests.cpp         |   4 +-
 extensions/gps/GetGPS.cpp                          |  26 +-
 extensions/gps/GetGPS.h                            |  39 +-
 extensions/http-curl/client/HTTPClient.cpp         |   3 +-
 extensions/http-curl/client/HTTPClient.h           |  13 +-
 extensions/http-curl/processors/InvokeHTTP.cpp     |  92 ++---
 extensions/http-curl/processors/InvokeHTTP.h       | 118 ++++--
 extensions/http-curl/protocols/AgentPrinter.cpp    |  19 +-
 extensions/http-curl/protocols/AgentPrinter.h      |  15 +-
 extensions/http-curl/protocols/RESTSender.cpp      |   3 +-
 extensions/http-curl/protocols/RESTSender.h        |   3 +-
 extensions/http-curl/sitetosite/HTTPProtocol.cpp   |  15 +-
 extensions/http-curl/sitetosite/HTTPProtocol.h     |  28 +-
 extensions/http-curl/tests/AbsoluteTimeoutTest.cpp |   5 +-
 .../tests/ControllerServiceIntegrationTests.cpp    |   4 +-
 extensions/http-curl/tests/HTTPHandlers.h          |   3 +-
 .../http-curl/tests/unit/InvokeHTTPTests.cpp       |  18 +-
 extensions/jni/ExecuteJavaControllerService.cpp    |  14 +-
 extensions/jni/ExecuteJavaControllerService.h      |  23 +-
 extensions/jni/ExecuteJavaProcessor.cpp            |  18 +-
 extensions/jni/ExecuteJavaProcessor.h              |  20 +-
 extensions/jni/JVMCreator.cpp                      |  15 +-
 extensions/jni/JVMCreator.h                        |  19 +-
 extensions/jni/jvm/JavaControllerService.cpp       |  22 +-
 extensions/jni/jvm/JavaControllerService.h         |  22 +-
 extensions/jni/jvm/JniConfigurationContext.h       |   7 +-
 extensions/jni/jvm/NarClassLoader.h                |  17 +-
 .../KubernetesControllerService.cpp                |   5 +-
 .../KubernetesControllerService.h                  |  16 +-
 extensions/libarchive/ArchiveStreamProvider.cpp    |   8 +-
 extensions/libarchive/BinFiles.cpp                 |  67 +---
 extensions/libarchive/BinFiles.h                   |  77 ++--
 .../libarchive/BinFilesStaticDefinitions.cpp       | 107 ++++++
 extensions/libarchive/CompressContent.cpp          |  36 +-
 extensions/libarchive/CompressContent.h            |  54 +--
 extensions/libarchive/FocusArchiveEntry.cpp        |  19 +-
 extensions/libarchive/FocusArchiveEntry.h          |  38 +-
 extensions/libarchive/ManipulateArchive.cpp        |  35 +-
 extensions/libarchive/ManipulateArchive.h          |  42 ++-
 extensions/libarchive/MergeContent.cpp             |  85 +----
 extensions/libarchive/MergeContent.h               |  95 +++--
 extensions/libarchive/UnfocusArchiveEntry.cpp      |  14 +-
 extensions/libarchive/UnfocusArchiveEntry.h        |  30 +-
 extensions/librdkafka/ConsumeKafka.cpp             | 179 +--------
 extensions/librdkafka/ConsumeKafka.h               | 106 +++---
 extensions/librdkafka/KafkaProcessorBase.cpp       |  43 ---
 extensions/librdkafka/KafkaProcessorBase.h         |  16 +-
 .../librdkafka/KafkaProcessorStaticDefinitions.cpp | 277 ++++++++++++++
 extensions/librdkafka/PublishKafka.cpp             | 151 +-------
 extensions/librdkafka/PublishKafka.h               |  72 ++--
 .../controllerservice/MQTTControllerService.cpp    |  47 +--
 .../mqtt/controllerservice/MQTTControllerService.h |  58 +--
 .../mqtt/processors/AbstractMQTTProcessor.cpp      |  36 +-
 extensions/mqtt/processors/AbstractMQTTProcessor.h |  83 ++---
 .../AbstractMQTTProcessorStaticDefinitions.cpp     |  65 ++++
 extensions/mqtt/processors/ConsumeMQTT.cpp         |  33 +-
 extensions/mqtt/processors/ConsumeMQTT.h           |  62 ++--
 extensions/mqtt/processors/ConvertBase.cpp         |  37 +-
 extensions/mqtt/processors/ConvertBase.h           |  45 +--
 .../processors/ConvertBaseStaticDefinitions.cpp    |  51 +++
 extensions/mqtt/processors/ConvertHeartBeat.cpp    |  10 +-
 extensions/mqtt/processors/ConvertHeartBeat.h      |  23 +-
 extensions/mqtt/processors/ConvertJSONAck.cpp      |   4 +-
 extensions/mqtt/processors/ConvertJSONAck.h        |  28 +-
 extensions/mqtt/processors/ConvertUpdate.cpp       |  30 +-
 extensions/mqtt/processors/ConvertUpdate.h         |  42 +--
 extensions/mqtt/processors/PublishMQTT.cpp         |  34 +-
 extensions/mqtt/processors/PublishMQTT.h           |  63 ++--
 extensions/mqtt/protocol/MQTTC2Protocol.cpp        |  15 +-
 extensions/mqtt/protocol/MQTTC2Protocol.h          |  25 +-
 extensions/opc/include/fetchopc.h                  |  61 +--
 extensions/opc/include/opc.h                       |   3 +
 extensions/opc/include/opcbase.h                   |  47 +--
 extensions/opc/include/putopc.h                    |  48 ++-
 extensions/opc/src/fetchopc.cpp                    |  68 +---
 extensions/opc/src/opc.cpp                         |   8 +
 extensions/opc/src/opcbase.cpp                     |  45 +--
 extensions/opc/src/opcbaseStaticDefinitions.cpp    | 149 ++++++++
 extensions/opc/src/putopc.cpp                      |  70 +---
 extensions/opencv/CaptureRTSPFrame.cpp             |  41 +--
 extensions/opencv/CaptureRTSPFrame.h               |  36 +-
 extensions/opencv/MotionDetector.cpp               |  29 +-
 extensions/opencv/MotionDetector.h                 |  35 +-
 .../SourceInitiatedSubscriptionListener.cpp        |  64 +---
 .../SourceInitiatedSubscriptionListener.h          |  72 ++--
 extensions/pcap/CapturePacket.cpp                  |  34 +-
 extensions/pcap/CapturePacket.h                    |  40 +-
 extensions/pdh/PerformanceDataMonitor.cpp          |  22 +-
 extensions/pdh/PerformanceDataMonitor.h            |  45 ++-
 extensions/procfs/processors/ProcFsMonitor.cpp     |   7 +-
 extensions/procfs/processors/ProcFsMonitor.h       |  25 +-
 .../rocksdb-repos/DatabaseContentRepository.cpp    |  25 +-
 .../rocksdb-repos/DatabaseContentRepository.h      |  25 +-
 extensions/rocksdb-repos/FlowFileRepository.cpp    |  17 +-
 extensions/rocksdb-repos/FlowFileRepository.h      |  21 +-
 extensions/rocksdb-repos/ProvenanceRepository.cpp  |  16 +-
 extensions/rocksdb-repos/ProvenanceRepository.h    |  21 +-
 .../RocksDbPersistableKeyValueStoreService.cpp     |  48 ++-
 .../RocksDbPersistableKeyValueStoreService.h       |  30 +-
 extensions/script/ExecuteScript.cpp                |  35 +-
 extensions/script/ExecuteScript.h                  |  40 +-
 .../script/python/ExecutePythonProcessor.cpp       |  19 +-
 extensions/script/python/ExecutePythonProcessor.h  |  32 +-
 extensions/script/python/PythonCreator.h           |  47 ++-
 .../script/tests/ExecutePythonProcessorTests.cpp   |   8 +-
 extensions/sensors/GetEnvironmentalSensors.cpp     |  18 +-
 extensions/sensors/GetEnvironmentalSensors.h       |  21 +-
 extensions/sensors/GetMovementSensors.cpp          |  16 +-
 extensions/sensors/GetMovementSensors.h            |  21 +-
 extensions/sensors/SensorBase.cpp                  |   2 -
 extensions/sensors/SensorBase.h                    |   2 -
 extensions/sftp/processors/FetchSFTP.cpp           |  90 +----
 extensions/sftp/processors/FetchSFTP.h             |  73 ++--
 extensions/sftp/processors/ListSFTP.cpp            | 136 +------
 extensions/sftp/processors/ListSFTP.h              |  92 ++---
 extensions/sftp/processors/PutSFTP.cpp             | 127 +------
 extensions/sftp/processors/PutSFTP.h               |  92 ++---
 extensions/sftp/processors/SFTPProcessorBase.cpp   | 101 +----
 extensions/sftp/processors/SFTPProcessorBase.h     |  61 +--
 .../processors/SFTPProcessorStaticDefinitions.cpp  | 325 ++++++++++++++++
 extensions/splunk/PutSplunkHTTP.cpp                |  36 +-
 extensions/splunk/PutSplunkHTTP.h                  |  24 +-
 extensions/splunk/QuerySplunkIndexingStatus.cpp    |  34 +-
 extensions/splunk/QuerySplunkIndexingStatus.h      |  28 +-
 extensions/splunk/SplunkHECProcessor.cpp           |  23 +-
 extensions/splunk/SplunkHECProcessor.h             |  13 +-
 .../splunk/SplunkHECProcessorStaticDefinitions.cpp | 114 ++++++
 extensions/splunk/tests/PutSplunkHTTPTests.cpp     |   6 +-
 .../tests/QuerySplunkIndexingStatusTests.cpp       |  10 +-
 extensions/sql/processors/ExecuteSQL.cpp           |  37 +-
 extensions/sql/processors/ExecuteSQL.h             |  24 +-
 extensions/sql/processors/FlowFileSource.cpp       |   1 +
 extensions/sql/processors/FlowFileSource.h         |   3 +
 extensions/sql/processors/PutSQL.cpp               |  25 +-
 extensions/sql/processors/PutSQL.h                 |  22 +-
 extensions/sql/processors/QueryDatabaseTable.cpp   |  67 +---
 extensions/sql/processors/QueryDatabaseTable.h     |  31 +-
 extensions/sql/processors/SQLProcessor.cpp         |  18 +-
 extensions/sql/processors/SQLProcessor.h           |   7 +-
 .../processors/SQLProcessorStaticDefinitions.cpp   | 113 ++++++
 extensions/sql/services/DatabaseService.cpp        |  24 +-
 extensions/sql/services/DatabaseService.h          |  16 +-
 ...or.cpp => DatabaseServiceStaticDefinitions.cpp} |  49 +--
 extensions/sql/services/ODBCConnector.cpp          |  27 +-
 extensions/sql/services/ODBCConnector.h            |  20 +-
 .../UnorderedMapKeyValueStoreService.cpp           |  25 +-
 .../controllers/UnorderedMapKeyValueStoreService.h |  24 +-
 ...UnorderedMapPersistableKeyValueStoreService.cpp |  44 ++-
 .../UnorderedMapPersistableKeyValueStoreService.h  |  34 +-
 .../processors/AppendHostInfo.cpp                  |  18 +-
 .../processors/AppendHostInfo.h                    |  30 +-
 .../processors/AttributesToJSON.cpp                |  14 +-
 .../processors/AttributesToJSON.h                  |  26 +-
 .../processors/DefragmentText.cpp                  |   8 +-
 .../processors/DefragmentText.h                    |  36 +-
 .../processors/ExecuteProcess.cpp                  |  20 +-
 .../processors/ExecuteProcess.h                    |  43 ++-
 .../standard-processors/processors/ExtractText.cpp |  20 +-
 .../standard-processors/processors/ExtractText.h   |  36 +-
 .../standard-processors/processors/FetchFile.cpp   |  21 +-
 .../standard-processors/processors/FetchFile.h     |  32 +-
 .../processors/GenerateFlowFile.cpp                |  30 +-
 .../processors/GenerateFlowFile.h                  |  53 +--
 .../standard-processors/processors/GetFile.cpp     |  48 +--
 .../standard-processors/processors/GetFile.h       |  79 ++--
 .../standard-processors/processors/GetTCP.cpp      |  42 +--
 extensions/standard-processors/processors/GetTCP.h |  79 ++--
 .../standard-processors/processors/HashContent.cpp |  27 +-
 .../standard-processors/processors/HashContent.h   |  47 +--
 .../standard-processors/processors/ListFile.cpp    |  21 +-
 .../standard-processors/processors/ListFile.h      |  28 +-
 .../processors/ListenSyslog.cpp                    |  11 +-
 .../standard-processors/processors/ListenSyslog.h  |  31 +-
 .../processors/LogAttribute.cpp                    |  39 +-
 .../standard-processors/processors/LogAttribute.h  |  59 +--
 .../standard-processors/processors/PutFile.cpp     |  60 ++-
 .../standard-processors/processors/PutFile.h       |  57 +--
 .../standard-processors/processors/PutUDP.cpp      |  14 +-
 extensions/standard-processors/processors/PutUDP.h |  15 +-
 .../standard-processors/processors/ReplaceText.cpp |  16 +-
 .../standard-processors/processors/ReplaceText.h   |  20 +-
 .../processors/RetryFlowFile.cpp                   |  41 +--
 .../standard-processors/processors/RetryFlowFile.h |  83 +++--
 .../processors/RouteOnAttribute.cpp                |  23 +-
 .../processors/RouteOnAttribute.h                  |  28 +-
 .../standard-processors/processors/RouteText.cpp   |  24 +-
 .../standard-processors/processors/RouteText.h     |  42 ++-
 .../standard-processors/processors/TailFile.cpp    |  23 +-
 .../standard-processors/processors/TailFile.h      |  47 ++-
 .../processors/UpdateAttribute.cpp                 |  14 +-
 .../processors/UpdateAttribute.h                   |  29 +-
 .../tests/unit/DefragmentTextTests.cpp             |  29 +-
 .../tests/unit/HashContentTest.cpp                 |   3 +-
 .../tests/unit/ManifestTests.cpp                   |  42 +--
 .../tests/unit/ProcessorTests.cpp                  |  35 +-
 .../tests/unit/PutFileTests.cpp                    |   8 +-
 .../tests/unit/RetryFlowFileTests.cpp              |  12 +-
 .../tests/unit/TailFileTests.cpp                   |   9 +-
 .../tests/unit/YamlConfigurationTests.cpp          |  56 +--
 extensions/systemd/ConsumeJournald.cpp             |   8 +-
 extensions/systemd/ConsumeJournald.h               |  39 +-
 extensions/tensorflow/TFApplyGraph.cpp             |  39 +-
 extensions/tensorflow/TFApplyGraph.h               |  51 ++-
 extensions/tensorflow/TFConvertImageToTensor.cpp   |  61 ++-
 extensions/tensorflow/TFConvertImageToTensor.h     |  65 ++--
 extensions/tensorflow/TFExtractTopLabels.cpp       |  30 +-
 extensions/tensorflow/TFExtractTopLabels.h         |  35 +-
 extensions/test-processors/KamikazeProcessor.cpp   |   9 +-
 extensions/test-processors/KamikazeProcessor.h     |  26 +-
 .../test-processors/LogOnDestructionProcessor.cpp  |   2 +-
 .../test-processors/LogOnDestructionProcessor.h    |   9 +
 extensions/usb-camera/GetUSBCamera.cpp             |  38 +-
 extensions/usb-camera/GetUSBCamera.h               |  42 ++-
 .../CollectorInitiatedSubscription.cpp             | 410 +++++++++++----------
 .../CollectorInitiatedSubscription.h               |  94 +++--
 .../windows-event-log/ConsumeWindowsEventLog.cpp   |  47 +--
 .../windows-event-log/ConsumeWindowsEventLog.h     |  90 +++--
 extensions/windows-event-log/SupportedProperty.h   | 110 ------
 extensions/windows-event-log/TailEventLog.cpp      |  23 +-
 extensions/windows-event-log/TailEventLog.h        |  35 +-
 libminifi/include/RemoteProcessorGroupPort.h       |  60 +--
 libminifi/include/agent/agent_docs.h               | 134 +++++--
 libminifi/include/agent/build_description.h        |  97 +----
 libminifi/include/c2/ControllerSocketProtocol.h    |  20 +-
 libminifi/include/c2/HeartbeatLogger.h             |  14 +-
 libminifi/include/c2/triggers/FileUpdateTrigger.h  |  36 +-
 .../controllers/LinuxPowerManagementService.h      |  65 ++--
 .../controllers/NetworkPrioritizerService.h        |  56 +--
 libminifi/include/controllers/SSLContextService.h  |  58 +--
 .../controllers/UpdatePolicyControllerService.h    |  40 +-
 .../AbstractAutoPersistingKeyValueStoreService.h   |  23 +-
 libminifi/include/core/Annotation.h                |  57 +--
 libminifi/include/core/ConfigurableComponent.h     |  42 +--
 libminifi/include/core/Connectable.h               |  35 +-
 libminifi/include/core/Funnel.h                    |  28 +-
 libminifi/include/core/ProcessContext.h            |  21 +-
 libminifi/include/core/ProcessContextBuilder.h     |  23 +-
 libminifi/include/core/Processor.h                 | 101 ++---
 libminifi/include/core/ProcessorNode.h             |  33 +-
 libminifi/include/core/Property.h                  | 204 +---------
 libminifi/include/core/PropertyBuilder.h           | 225 +++++++++++
 libminifi/include/core/Resource.h                  |  58 +--
 .../include/core/controller/ControllerService.h    |  23 +-
 .../core/controller/ControllerServiceNode.h        |  20 +-
 .../core/controller/ControllerServiceProvider.h    |   6 +-
 .../include/core/state/nodes/AgentInformation.h    |  68 ++--
 .../include/core/state/nodes/BuildInformation.h    |  28 +-
 .../core/state/nodes/ConfigurationChecksums.h      |  16 +-
 .../include/core/state/nodes/DeviceInformation.h   |  28 +-
 .../include/core/state/nodes/FlowInformation.h     |  36 +-
 .../include/core/state/nodes/SupportedOperations.h |   3 +-
 .../agent_docs.cpp => include/utils/ArrayUtils.h}  |  31 +-
 libminifi/src/Configuration.cpp                    |   2 +-
 libminifi/src/RemoteProcessorGroupPort.cpp         |  30 +-
 libminifi/src/agent/agent_docs.cpp                 |  17 +-
 libminifi/src/c2/ControllerSocketProtocol.cpp      |  15 +-
 libminifi/src/c2/HeartbeatLogger.cpp               |  14 +-
 libminifi/src/c2/triggers/FileUpdateTrigger.cpp    |  15 +-
 .../controllers/LinuxPowerManagementService.cpp    |  37 +-
 .../src/controllers/NetworkPrioritizerService.cpp  |  35 +-
 libminifi/src/controllers/SSLContextService.cpp    |  32 +-
 .../controllers/UpdatePolicyControllerService.cpp  |  33 +-
 .../AbstractAutoPersistingKeyValueStoreService.cpp |  31 +-
 libminifi/src/core/ConfigurableComponent.cpp       |  26 +-
 libminifi/src/core/Connectable.cpp                 |  16 +-
 libminifi/src/core/Funnel.cpp                      |  16 +-
 libminifi/src/core/ProcessContextBuilder.cpp       |  25 +-
 libminifi/src/core/Processor.cpp                   |  13 -
 .../src/core/state/nodes/AgentInformation.cpp      |   2 +-
 .../src/core/state/nodes/BuildInformation.cpp      |   2 +-
 .../core/state/nodes/ConfigurationChecksums.cpp    |  16 +-
 .../src/core/state/nodes/DeviceInformation.cpp     |  17 +-
 libminifi/src/core/state/nodes/FlowInformation.cpp |  17 +-
 .../src/core/state/nodes/SupportedOperations.cpp   |   4 +-
 libminifi/src/core/yaml/YamlConfiguration.cpp      |   4 +-
 libminifi/test/ReadFromFlowFileTestProcessor.cpp   |   4 +-
 libminifi/test/ReadFromFlowFileTestProcessor.h     |  13 +-
 libminifi/test/StatefulProcessor.cpp               |  15 +-
 libminifi/test/StatefulProcessor.h                 |  25 +-
 libminifi/test/WriteToFlowFileTestProcessor.cpp    |   6 +-
 libminifi/test/WriteToFlowFileTestProcessor.h      |  10 +-
 libminifi/test/archive-tests/MergeFileTests.cpp    |   4 +-
 libminifi/test/aws-tests/DeleteS3ObjectTests.cpp   |   3 +-
 .../azure-tests/AzureBlobStorageTestsFixture.h     |   5 +-
 .../azure-tests/AzureDataLakeStorageTestsFixture.h |   5 +-
 libminifi/test/flow-tests/CustomProcessors.h       |  22 +-
 libminifi/test/flow-tests/SessionTests.cpp         |  13 +-
 .../test/persistence-tests/PersistenceTests.cpp    |  27 +-
 libminifi/test/rocksdb-tests/RepoTests.cpp         |  15 +-
 libminifi/test/sql-tests/mocks/MockODBCService.h   |  24 +-
 libminifi/test/unit/ComponentManifestTests.cpp     |  35 +-
 .../test/unit/ContentRepositoryDependentTests.h    |  12 +-
 libminifi/test/unit/DynamicPropertyTests.cpp       |  16 +-
 libminifi/test/unit/MockClasses.h                  |  51 ++-
 libminifi/test/unit/ProcessSessionTests.cpp        |  13 +-
 libminifi/test/unit/ProcessorConfigUtilsTests.cpp  |  10 +-
 libminifi/test/unit/PropertyValidationTests.cpp    |  44 +--
 libminifi/test/unit/SchedulingAgentTests.cpp       |   6 +
 main/AgentDocs.cpp                                 |  28 +-
 nanofi/include/cxx/CallbackProcessor.h             |  34 +-
 nanofi/src/cxx/CallbackProcessor.cpp               |  11 +-
 376 files changed, 6986 insertions(+), 7574 deletions(-)

diff --git a/CONTRIB.md b/CONTRIB.md
index d5fc45369..482955001 100644
--- a/CONTRIB.md
+++ b/CONTRIB.md
@@ -99,9 +99,11 @@ Please see [ThirdParties.md](ThirdParties.md) on how MiNiFi builds and uses thir
 MiNiFi C++ contains a dynamic loading mechanism that loads arbitrary objects. To maintain consistency of development amongst the NiFi ecosystem, it is called a class loader. If you
 are contributing a custom Processor or Controller Service, the mechanism to register your class into the default class loader is a pragma definition named:
 
-    REGISTER_RESOURCE(CLASSNAME,DOCUMENTATION);
+    REGISTER_RESOURCE(CLASSNAME, TYPE);
 
-To use this include REGISTER_RESOURCE(YourClassName); in your header file. The default class loader will make instances of YourClassName available for inclusion.  
+To use this include REGISTER_RESOURCE(YourClassName, Processor) or REGISTER_RESOURCE(YourClassName, ControllerService) in your cpp file. The default class loader will make instances
+of YourClassName available for inclusion. In order to include your new component in the manifest in the heartbeat message in a standardized way, REGISTER_RESOURCE requires the
+presence of a number of static variables and functions. Use an existing Processor or ControllerService as a model to create these static members.
 
 The extensions sub-directory allows you to contribute conditionally built extensions. The system adds all subdirectories in `extensions/*` that contain
 a `CMakeLists.txt` file. It is up to the extension creator's discretion how they handle cmake flags.
diff --git a/Extensions.md b/Extensions.md
index 212de7e77..0e9b94626 100644
--- a/Extensions.md
+++ b/Extensions.md
@@ -20,14 +20,16 @@ To enable all extensions for your platform, you may use -DENABLE_ALL=TRUE OR sel
 Extensions are dynamic libraries loaded at runtime by the agent. An extension makes its 
 capabilities (classes) available to the system through registrars. Registration must happen in source files, not headers.
 
-``` C++
+```C++
 // register user-facing classes as
-REGISTER_RESOURCE(InvokeHTTP, "An HTTP client processor which can interact with a configurable HTTP Endpoint. "
-    "The destination URL and HTTP Method are configurable. FlowFile attributes are converted to HTTP headers and the "
-    "FlowFile contents are included as the body of the request (if the HTTP Method is PUT, POST or PATCH).");
+REGISTER_RESOURCE(InvokeHTTP, Processor);
+// or
+REGISTER_RESOURCE(SSLContextService, ControllerService);
 
 // register internal resources as
-REGISTER_INTERNAL_RESOURCE(HTTPClient);
+REGISTER_RESOURCE(HTTPClient, InternalResource);
+// or
+REGISTER_RESOURCE(RESTSender, DescriptionOnly);
 ```
 
 Some extensions (e.g. `http-curl`) require initialization before use. 
diff --git a/extensions/aws/controllerservices/AWSCredentialsService.cpp b/extensions/aws/controllerservices/AWSCredentialsService.cpp
index ec5369d37..afe7fdf47 100644
--- a/extensions/aws/controllerservices/AWSCredentialsService.cpp
+++ b/extensions/aws/controllerservices/AWSCredentialsService.cpp
@@ -18,8 +18,7 @@
 
 #include "AWSCredentialsService.h"
 
-#include <set>
-
+#include "core/PropertyBuilder.h"
 #include "core/Resource.h"
 
 namespace org {
@@ -51,12 +50,7 @@ const core::Property AWSCredentialsService::CredentialsFile(
     ->build());
 
 void AWSCredentialsService::initialize() {
-  std::set<core::Property> supportedProperties;
-  supportedProperties.insert(AccessKey);
-  supportedProperties.insert(SecretKey);
-  supportedProperties.insert(UseDefaultCredentials);
-  supportedProperties.insert(CredentialsFile);
-  setSupportedProperties(supportedProperties);
+  setSupportedProperties(properties());
 }
 
 void AWSCredentialsService::onEnable() {
@@ -83,7 +77,7 @@ std::optional<Aws::Auth::AWSCredentials> AWSCredentialsService::getAWSCredential
   return aws_credentials_;
 }
 
-REGISTER_RESOURCE(AWSCredentialsService, "AWS Credentials Management Service");
+REGISTER_RESOURCE(AWSCredentialsService, ControllerService);
 
 }  // namespace controllers
 }  // namespace aws
diff --git a/extensions/aws/controllerservices/AWSCredentialsService.h b/extensions/aws/controllerservices/AWSCredentialsService.h
index 15994063f..9d40076e0 100644
--- a/extensions/aws/controllerservices/AWSCredentialsService.h
+++ b/extensions/aws/controllerservices/AWSCredentialsService.h
@@ -40,11 +40,6 @@ namespace controllers {
 
 class AWSCredentialsService : public core::controller::ControllerService {
  public:
-  static const core::Property UseDefaultCredentials;
-  static const core::Property AccessKey;
-  static const core::Property SecretKey;
-  static const core::Property CredentialsFile;
-
   explicit AWSCredentialsService(const std::string &name, const minifi::utils::Identifier &uuid = {})
       : ControllerService(name, uuid) {
   }
@@ -53,6 +48,24 @@ class AWSCredentialsService : public core::controller::ControllerService {
       : ControllerService(name) {
   }
 
+  EXTENSIONAPI static constexpr const char* Description = "AWS Credentials Management Service";
+
+  EXTENSIONAPI static const core::Property UseDefaultCredentials;
+  EXTENSIONAPI static const core::Property AccessKey;
+  EXTENSIONAPI static const core::Property SecretKey;
+  EXTENSIONAPI static const core::Property CredentialsFile;
+  static auto properties() {
+    return std::array{
+      UseDefaultCredentials,
+      AccessKey,
+      SecretKey,
+      CredentialsFile
+    };
+  }
+
+  EXTENSIONAPI static constexpr bool SupportsDynamicProperties = false;
+  ADD_COMMON_VIRTUAL_FUNCTIONS_FOR_CONTROLLER_SERVICES
+
   void initialize() override;
 
   void yield() override {
diff --git a/extensions/aws/processors/DeleteS3Object.cpp b/extensions/aws/processors/DeleteS3Object.cpp
index 05d4f1468..c9a2bc73b 100644
--- a/extensions/aws/processors/DeleteS3Object.cpp
+++ b/extensions/aws/processors/DeleteS3Object.cpp
@@ -1,7 +1,4 @@
 /**
- * @file DeleteS3Object.cpp
- * DeleteS3Object class implementation
- *
  * 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.
@@ -27,33 +24,11 @@
 #include "core/ProcessSession.h"
 #include "core/Resource.h"
 
-namespace org {
-namespace apache {
-namespace nifi {
-namespace minifi {
-namespace aws {
-namespace processors {
-
-const core::Property DeleteS3Object::ObjectKey(
-  core::PropertyBuilder::createProperty("Object Key")
-    ->withDescription("The key of the S3 object. If none is given the filename attribute will be used by default.")
-    ->supportsExpressionLanguage(true)
-    ->build());
-const core::Property DeleteS3Object::Version(
-  core::PropertyBuilder::createProperty("Version")
-    ->withDescription("The Version of the Object to delete")
-    ->supportsExpressionLanguage(true)
-    ->build());
-
-const core::Relationship DeleteS3Object::Success("success", "FlowFiles are routed to success relationship");
-const core::Relationship DeleteS3Object::Failure("failure", "FlowFiles are routed to failure relationship");
+namespace org::apache::nifi::minifi::aws::processors {
 
 void DeleteS3Object::initialize() {
-  // Add new supported properties
-  setSupportedProperties({Bucket, AccessKey, SecretKey, CredentialsFile, CredentialsFile, AWSCredentialsProviderService, Region, CommunicationsTimeout,
-                          EndpointOverrideURL, ProxyHost, ProxyPort, ProxyUsername, ProxyPassword, UseDefaultCredentials, ObjectKey, Version});
-  // Set the supported relationships
-  setSupportedRelationships({Failure, Success});
+  setSupportedProperties(properties());
+  setSupportedRelationships(relationships());
 }
 
 std::optional<aws::s3::DeleteObjectRequestParameters> DeleteS3Object::buildDeleteS3RequestParams(
@@ -106,11 +81,4 @@ void DeleteS3Object::onTrigger(const std::shared_ptr<core::ProcessContext> &cont
   }
 }
 
-REGISTER_RESOURCE(DeleteS3Object, "This Processor deletes FlowFiles on an Amazon S3 Bucket.");
-
-}  // namespace processors
-}  // namespace aws
-}  // namespace minifi
-}  // namespace nifi
-}  // namespace apache
-}  // namespace org
+}  // namespace org::apache::nifi::minifi::aws::processors
diff --git a/extensions/aws/processors/DeleteS3Object.h b/extensions/aws/processors/DeleteS3Object.h
index 55c69cde2..1566eb86a 100644
--- a/extensions/aws/processors/DeleteS3Object.h
+++ b/extensions/aws/processors/DeleteS3Object.h
@@ -1,7 +1,4 @@
 /**
- * @file DeleteS3Object.h
- * DeleteS3Object class declaration
- *
  * 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.
@@ -27,29 +24,37 @@
 #include <utility>
 
 #include "S3Processor.h"
+#include "utils/ArrayUtils.h"
 #include "utils/GeneralUtils.h"
 
 template<typename T>
 class S3TestsFixture;
 
-namespace org {
-namespace apache {
-namespace nifi {
-namespace minifi {
-namespace aws {
-namespace processors {
+namespace org::apache::nifi::minifi::aws::processors {
 
 class DeleteS3Object : public S3Processor {
  public:
-  static constexpr char const* ProcessorName = "DeleteS3Object";
+  EXTENSIONAPI static constexpr const char* Description = "This Processor deletes FlowFiles on an Amazon S3 Bucket.";
 
-  // Supported Properties
   static const core::Property ObjectKey;
   static const core::Property Version;
+  static auto properties() {
+    return minifi::utils::array_cat(S3Processor::properties(), std::array{
+      ObjectKey,
+      Version
+    });
+  }
+
+  EXTENSIONAPI static const core::Relationship Success;
+  EXTENSIONAPI static const core::Relationship Failure;
+  static auto relationships() { return std::array{Success, Failure}; }
+
+  EXTENSIONAPI static constexpr bool SupportsDynamicProperties = true;
+  EXTENSIONAPI static constexpr bool SupportsDynamicRelationships = false;
+  EXTENSIONAPI static constexpr core::annotation::Input InputRequirement = core::annotation::Input::INPUT_REQUIRED;
+  EXTENSIONAPI static constexpr bool IsSingleThreaded = true;
 
-  // Supported Relationships
-  static const core::Relationship Failure;
-  static const core::Relationship Success;
+  ADD_COMMON_VIRTUAL_FUNCTIONS_FOR_PROCESSORS
 
   explicit DeleteS3Object(const std::string& name, const minifi::utils::Identifier& uuid = minifi::utils::Identifier())
     : S3Processor(name, uuid, core::logging::LoggerFactory<DeleteS3Object>::getLogger()) {
@@ -61,14 +66,6 @@ class DeleteS3Object : public S3Processor {
   void onTrigger(const std::shared_ptr<core::ProcessContext> &context, const std::shared_ptr<core::ProcessSession> &session) override;
 
  private:
-  core::annotation::Input getInputRequirement() const override {
-    return core::annotation::Input::INPUT_REQUIRED;
-  }
-
-  bool isSingleThreaded() const override {
-    return true;
-  }
-
   friend class ::S3TestsFixture<DeleteS3Object>;
 
   explicit DeleteS3Object(const std::string& name, const minifi::utils::Identifier& uuid, std::unique_ptr<aws::s3::S3RequestSender> s3_request_sender)
@@ -81,9 +78,4 @@ class DeleteS3Object : public S3Processor {
     const CommonProperties &common_properties) const;
 };
 
-}  // namespace processors
-}  // namespace aws
-}  // namespace minifi
-}  // namespace nifi
-}  // namespace apache
-}  // namespace org
+}  // namespace org::apache::nifi::minifi::aws::processors
diff --git a/extensions/aws/processors/FetchS3Object.cpp b/extensions/aws/processors/FetchS3Object.cpp
index af93ebe55..9ee0fd27a 100644
--- a/extensions/aws/processors/FetchS3Object.cpp
+++ b/extensions/aws/processors/FetchS3Object.cpp
@@ -35,33 +35,9 @@ namespace minifi {
 namespace aws {
 namespace processors {
 
-const core::Property FetchS3Object::ObjectKey(
-  core::PropertyBuilder::createProperty("Object Key")
-    ->withDescription("The key of the S3 object. If none is given the filename attribute will be used by default.")
-    ->supportsExpressionLanguage(true)
-    ->build());
-const core::Property FetchS3Object::Version(
-  core::PropertyBuilder::createProperty("Version")
-    ->withDescription("The Version of the Object to download")
-    ->supportsExpressionLanguage(true)
-    ->build());
-const core::Property FetchS3Object::RequesterPays(
-  core::PropertyBuilder::createProperty("Requester Pays")
-    ->isRequired(true)
-    ->withDefaultValue<bool>(false)
-    ->withDescription("If true, indicates that the requester consents to pay any charges associated with retrieving "
-                      "objects from the S3 bucket. This sets the 'x-amz-request-payer' header to 'requester'.")
-    ->build());
-
-const core::Relationship FetchS3Object::Success("success", "FlowFiles are routed to success relationship");
-const core::Relationship FetchS3Object::Failure("failure", "FlowFiles are routed to failure relationship");
-
 void FetchS3Object::initialize() {
-  // Add new supported properties
-  setSupportedProperties({Bucket, AccessKey, SecretKey, CredentialsFile, CredentialsFile, AWSCredentialsProviderService, Region, CommunicationsTimeout,
-                          EndpointOverrideURL, ProxyHost, ProxyPort, ProxyUsername, ProxyPassword, UseDefaultCredentials, ObjectKey, Version, RequesterPays});
-  // Set the supported relationships
-  setSupportedRelationships({Failure, Success});
+  setSupportedProperties(properties());
+  setSupportedRelationships(relationships());
 }
 
 void FetchS3Object::onSchedule(const std::shared_ptr<core::ProcessContext> &context, const std::shared_ptr<core::ProcessSessionFactory> &sessionFactory) {
@@ -144,8 +120,6 @@ void FetchS3Object::onTrigger(const std::shared_ptr<core::ProcessContext> &conte
   }
 }
 
-REGISTER_RESOURCE(FetchS3Object, "This Processor retrieves the contents of an S3 Object and writes it to the content of a FlowFile.");
-
 }  // namespace processors
 }  // namespace aws
 }  // namespace minifi
diff --git a/extensions/aws/processors/FetchS3Object.h b/extensions/aws/processors/FetchS3Object.h
index 4c16e5e71..5a5b73344 100644
--- a/extensions/aws/processors/FetchS3Object.h
+++ b/extensions/aws/processors/FetchS3Object.h
@@ -1,7 +1,4 @@
 /**
- * @file FetchS3Object.h
- * FetchS3Object class declaration
- *
  * 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.
@@ -29,30 +26,39 @@
 
 #include "io/StreamPipe.h"
 #include "S3Processor.h"
+#include "utils/ArrayUtils.h"
 #include "utils/GeneralUtils.h"
 
 template<typename T>
 class S3TestsFixture;
 
-namespace org {
-namespace apache {
-namespace nifi {
-namespace minifi {
-namespace aws {
-namespace processors {
+namespace org::apache::nifi::minifi::aws::processors {
 
 class FetchS3Object : public S3Processor {
  public:
-  static constexpr char const* ProcessorName = "FetchS3Object";
+  EXTENSIONAPI static constexpr const char* Description = "This Processor retrieves the contents of an S3 Object and writes it to the content of a FlowFile.";
 
-  // Supported Properties
   static const core::Property ObjectKey;
   static const core::Property Version;
   static const core::Property RequesterPays;
+  static auto properties() {
+    return minifi::utils::array_cat(S3Processor::properties(), std::array{
+      ObjectKey,
+      Version,
+      RequesterPays
+    });
+  }
+
+  EXTENSIONAPI static const core::Relationship Success;
+  EXTENSIONAPI static const core::Relationship Failure;
+  static auto relationships() { return std::array{Success, Failure}; }
+
+  EXTENSIONAPI static constexpr bool SupportsDynamicProperties = true;
+  EXTENSIONAPI static constexpr bool SupportsDynamicRelationships = false;
+  EXTENSIONAPI static constexpr core::annotation::Input InputRequirement = core::annotation::Input::INPUT_REQUIRED;
+  EXTENSIONAPI static constexpr bool IsSingleThreaded = true;
 
-  // Supported Relationships
-  static const core::Relationship Failure;
-  static const core::Relationship Success;
+  ADD_COMMON_VIRTUAL_FUNCTIONS_FOR_PROCESSORS
 
   explicit FetchS3Object(const std::string& name, const minifi::utils::Identifier& uuid = minifi::utils::Identifier())
     : S3Processor(name, uuid, core::logging::LoggerFactory<FetchS3Object>::getLogger()) {
@@ -65,14 +71,6 @@ class FetchS3Object : public S3Processor {
   void onTrigger(const std::shared_ptr<core::ProcessContext> &context, const std::shared_ptr<core::ProcessSession> &session) override;
 
  private:
-  core::annotation::Input getInputRequirement() const override {
-    return core::annotation::Input::INPUT_REQUIRED;
-  }
-
-  bool isSingleThreaded() const override {
-    return true;
-  }
-
   friend class ::S3TestsFixture<FetchS3Object>;
 
   explicit FetchS3Object(const std::string& name, const minifi::utils::Identifier& uuid, std::unique_ptr<aws::s3::S3RequestSender> s3_request_sender)
@@ -87,9 +85,4 @@ class FetchS3Object : public S3Processor {
   bool requester_pays_ = false;
 };
 
-}  // namespace processors
-}  // namespace aws
-}  // namespace minifi
-}  // namespace nifi
-}  // namespace apache
-}  // namespace org
+}  // namespace org::apache::nifi::minifi::aws::processors
diff --git a/extensions/aws/processors/ListS3.cpp b/extensions/aws/processors/ListS3.cpp
index 1ffd420c9..3e22482b4 100644
--- a/extensions/aws/processors/ListS3.cpp
+++ b/extensions/aws/processors/ListS3.cpp
@@ -1,7 +1,4 @@
 /**
- * @file ListS3.cpp
- * ListS3 class implementation
- *
  * 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.
@@ -28,62 +25,11 @@
 #include "core/ProcessSession.h"
 #include "core/Resource.h"
 
-namespace org {
-namespace apache {
-namespace nifi {
-namespace minifi {
-namespace aws {
-namespace processors {
-
-const core::Property ListS3::Delimiter(
-  core::PropertyBuilder::createProperty("Delimiter")
-    ->withDescription("The string used to delimit directories within the bucket. Please consult the AWS documentation for the correct use of this field.")
-    ->build());
-const core::Property ListS3::Prefix(
-  core::PropertyBuilder::createProperty("Prefix")
-    ->withDescription("The prefix used to filter the object list. In most cases, it should end with a forward slash ('/').")
-    ->build());
-const core::Property ListS3::UseVersions(
-  core::PropertyBuilder::createProperty("Use Versions")
-    ->isRequired(true)
-    ->withDefaultValue<bool>(false)
-    ->withDescription("Specifies whether to use S3 versions, if applicable. If false, only the latest version of each object will be returned.")
-    ->build());
-const core::Property ListS3::MinimumObjectAge(
-  core::PropertyBuilder::createProperty("Minimum Object Age")
-    ->isRequired(true)
-    ->withDefaultValue<core::TimePeriodValue>("0 sec")
-    ->withDescription("The minimum age that an S3 object must be in order to be considered; any object younger than this amount of time (according to last modification date) will be ignored.")
-    ->build());
-const core::Property ListS3::WriteObjectTags(
-  core::PropertyBuilder::createProperty("Write Object Tags")
-    ->isRequired(true)
-    ->withDefaultValue<bool>(false)
-    ->withDescription("If set to 'true', the tags associated with the S3 object will be written as FlowFile attributes.")
-    ->build());
-const core::Property ListS3::WriteUserMetadata(
-  core::PropertyBuilder::createProperty("Write User Metadata")
-    ->isRequired(true)
-    ->withDefaultValue<bool>(false)
-    ->withDescription("If set to 'true', the user defined metadata associated with the S3 object will be added to FlowFile attributes/records.")
-    ->build());
-const core::Property ListS3::RequesterPays(
-  core::PropertyBuilder::createProperty("Requester Pays")
-    ->isRequired(true)
-    ->withDefaultValue<bool>(false)
-    ->withDescription("If true, indicates that the requester consents to pay any charges associated with listing the S3 bucket. This sets the 'x-amz-request-payer' header to 'requester'. "
-                      "Note that this setting is only used if Write User Metadata is true.")
-    ->build());
-
-const core::Relationship ListS3::Success("success", "FlowFiles are routed to success relationship");
+namespace org::apache::nifi::minifi::aws::processors {
 
 void ListS3::initialize() {
-  // Add new supported properties
-  setSupportedProperties({Bucket, AccessKey, SecretKey, CredentialsFile, CredentialsFile, AWSCredentialsProviderService, Region, CommunicationsTimeout,
-                          EndpointOverrideURL, ProxyHost, ProxyPort, ProxyUsername, ProxyPassword, UseDefaultCredentials, Delimiter, Prefix, UseVersions,
-                          MinimumObjectAge, WriteObjectTags, WriteUserMetadata, RequesterPays});
-  // Set the supported relationships
-  setSupportedRelationships({Success});
+  setSupportedProperties(properties());
+  setSupportedRelationships(relationships());
 }
 
 void ListS3::onSchedule(const std::shared_ptr<core::ProcessContext> &context, const std::shared_ptr<core::ProcessSessionFactory> &sessionFactory) {
@@ -230,11 +176,4 @@ void ListS3::onTrigger(const std::shared_ptr<core::ProcessContext> &context, con
   }
 }
 
-REGISTER_RESOURCE(ListS3, "This Processor retrieves a listing of objects from an Amazon S3 bucket.");
-
-}  // namespace processors
-}  // namespace aws
-}  // namespace minifi
-}  // namespace nifi
-}  // namespace apache
-}  // namespace org
+}  // namespace org::apache::nifi::minifi::aws::processors
diff --git a/extensions/aws/processors/ListS3.h b/extensions/aws/processors/ListS3.h
index ff3a53c19..88a7c4d8d 100644
--- a/extensions/aws/processors/ListS3.h
+++ b/extensions/aws/processors/ListS3.h
@@ -1,7 +1,4 @@
 /**
- * @file ListS3.h
- * ListS3 class declaration
- *
  * 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.
@@ -28,29 +25,42 @@
 #include <memory>
 
 #include "S3Processor.h"
+#include "utils/ArrayUtils.h"
 
-namespace org {
-namespace apache {
-namespace nifi {
-namespace minifi {
-namespace aws {
-namespace processors {
+namespace org::apache::nifi::minifi::aws::processors {
 
 class ListS3 : public S3Processor {
  public:
-  static constexpr char const* ProcessorName = "ListS3";
+  EXTENSIONAPI static constexpr const char* Description = "This Processor retrieves a listing of objects from an Amazon S3 bucket.";
+
+  EXTENSIONAPI static const core::Property Delimiter;
+  EXTENSIONAPI static const core::Property Prefix;
+  EXTENSIONAPI static const core::Property UseVersions;
+  EXTENSIONAPI static const core::Property MinimumObjectAge;
+  EXTENSIONAPI static const core::Property WriteObjectTags;
+  EXTENSIONAPI static const core::Property WriteUserMetadata;
+  EXTENSIONAPI static const core::Property RequesterPays;
+  static auto properties() {
+    return minifi::utils::array_cat(S3Processor::properties(), std::array{
+      Delimiter,
+      Prefix,
+      UseVersions,
+      MinimumObjectAge,
+      WriteObjectTags,
+      WriteUserMetadata,
+      RequesterPays
+    });
+  }
 
-  // Supported Properties
-  static const core::Property Delimiter;
-  static const core::Property Prefix;
-  static const core::Property UseVersions;
-  static const core::Property MinimumObjectAge;
-  static const core::Property WriteObjectTags;
-  static const core::Property WriteUserMetadata;
-  static const core::Property RequesterPays;
+  EXTENSIONAPI static const core::Relationship Success;
+  static auto relationships() { return std::array{Success}; }
 
-  // Supported Relationships
-  static const core::Relationship Success;
+  EXTENSIONAPI static constexpr bool SupportsDynamicProperties = true;
+  EXTENSIONAPI static constexpr bool SupportsDynamicRelationships = false;
+  EXTENSIONAPI static constexpr core::annotation::Input InputRequirement = core::annotation::Input::INPUT_FORBIDDEN;
+  EXTENSIONAPI static constexpr bool IsSingleThreaded = false;
+
+  ADD_COMMON_VIRTUAL_FUNCTIONS_FOR_PROCESSORS
 
   explicit ListS3(const std::string& name, const minifi::utils::Identifier& uuid = minifi::utils::Identifier())
     : S3Processor(name, uuid, core::logging::LoggerFactory<ListS3>::getLogger()) {
@@ -66,10 +76,6 @@ class ListS3 : public S3Processor {
   void onTrigger(const std::shared_ptr<core::ProcessContext> &context, const std::shared_ptr<core::ProcessSession> &session) override;
 
  private:
-  core::annotation::Input getInputRequirement() const override {
-    return core::annotation::Input::INPUT_FORBIDDEN;
-  }
-
   void writeObjectTags(
     const aws::s3::ListedObjectAttributes &object_attributes,
     core::ProcessSession &session,
@@ -89,9 +95,4 @@ class ListS3 : public S3Processor {
   std::unique_ptr<minifi::utils::ListingStateManager> state_manager_;
 };
 
-}  // namespace processors
-}  // namespace aws
-}  // namespace minifi
-}  // namespace nifi
-}  // namespace apache
-}  // namespace org
+}  // namespace org::apache::nifi::minifi::aws::processors
diff --git a/extensions/aws/processors/PutS3Object.cpp b/extensions/aws/processors/PutS3Object.cpp
index 09b9ba337..6fe350d2a 100644
--- a/extensions/aws/processors/PutS3Object.cpp
+++ b/extensions/aws/processors/PutS3Object.cpp
@@ -1,7 +1,4 @@
 /**
- * @file PutS3Object.cpp
- * PutS3Object class implementation
- *
  * 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.
@@ -33,87 +30,11 @@
 #include "core/ProcessSession.h"
 #include "core/Resource.h"
 
-namespace org {
-namespace apache {
-namespace nifi {
-namespace minifi {
-namespace aws {
-namespace processors {
-
-const uint64_t PutS3Object::ReadCallback::MAX_SIZE = 5UL * 1024UL * 1024UL * 1024UL;  // 5GB limit on AWS
-const uint64_t PutS3Object::ReadCallback::BUFFER_SIZE = 4096;
-
-const std::set<std::string> PutS3Object::CANNED_ACLS(minifi::utils::MapUtils::getKeys(minifi::aws::s3::CANNED_ACL_MAP));
-const std::set<std::string> PutS3Object::STORAGE_CLASSES(minifi::utils::MapUtils::getKeys(minifi::aws::s3::STORAGE_CLASS_MAP));
-const std::set<std::string> PutS3Object::SERVER_SIDE_ENCRYPTIONS(minifi::utils::MapUtils::getKeys(minifi::aws::s3::SERVER_SIDE_ENCRYPTION_MAP));
-
-const core::Property PutS3Object::ObjectKey(
-  core::PropertyBuilder::createProperty("Object Key")
-    ->withDescription("The key of the S3 object. If none is given the filename attribute will be used by default.")
-    ->supportsExpressionLanguage(true)
-    ->build());
-const core::Property PutS3Object::ContentType(
-  core::PropertyBuilder::createProperty("Content Type")
-    ->withDescription("Sets the Content-Type HTTP header indicating the type of content stored in "
-                      "the associated object. The value of this header is a standard MIME type. "
-                      "If no content type is provided the default content type "
-                      "\"application/octet-stream\" will be used.")
-    ->supportsExpressionLanguage(true)
-    ->withDefaultValue<std::string>("application/octet-stream")
-    ->build());
-const core::Property PutS3Object::StorageClass(
-  core::PropertyBuilder::createProperty("Storage Class")
-    ->isRequired(true)
-    ->withDefaultValue<std::string>("Standard")
-    ->withAllowableValues<std::string>(PutS3Object::STORAGE_CLASSES)
-    ->withDescription("AWS S3 Storage Class")
-    ->build());
-const core::Property PutS3Object::FullControlUserList(
-  core::PropertyBuilder::createProperty("FullControl User List")
-    ->withDescription("A comma-separated list of Amazon User ID's or E-mail addresses that specifies who should have Full Control for an object.")
-    ->supportsExpressionLanguage(true)
-    ->build());
-const core::Property PutS3Object::ReadPermissionUserList(
-  core::PropertyBuilder::createProperty("Read Permission User List")
-    ->withDescription("A comma-separated list of Amazon User ID's or E-mail addresses that specifies who should have Read Access for an object.")
-    ->supportsExpressionLanguage(true)
-    ->build());
-const core::Property PutS3Object::ReadACLUserList(
-  core::PropertyBuilder::createProperty("Read ACL User List")
-    ->withDescription("A comma-separated list of Amazon User ID's or E-mail addresses that specifies who should have permissions to read "
-                      "the Access Control List for an object.")
-    ->supportsExpressionLanguage(true)
-    ->build());
-const core::Property PutS3Object::WriteACLUserList(
-  core::PropertyBuilder::createProperty("Write ACL User List")
-    ->withDescription("A comma-separated list of Amazon User ID's or E-mail addresses that specifies who should have permissions to change "
-                      "the Access Control List for an object.")
-    ->supportsExpressionLanguage(true)
-    ->build());
-const core::Property PutS3Object::CannedACL(
-  core::PropertyBuilder::createProperty("Canned ACL")
-    ->withDescription("Amazon Canned ACL for an object. Allowed values: BucketOwnerFullControl, BucketOwnerRead, AuthenticatedRead, "
-                      "PublicReadWrite, PublicRead, Private, AwsExecRead; will be ignored if any other ACL/permission property is specified.")
-    ->supportsExpressionLanguage(true)
-    ->build());
-const core::Property PutS3Object::ServerSideEncryption(
-  core::PropertyBuilder::createProperty("Server Side Encryption")
-    ->isRequired(true)
-    ->withDefaultValue<std::string>("None")
-    ->withAllowableValues<std::string>(PutS3Object::SERVER_SIDE_ENCRYPTIONS)
-    ->withDescription("Specifies the algorithm used for server side encryption.")
-    ->build());
-
-const core::Relationship PutS3Object::Success("success", "FlowFiles are routed to success relationship");
-const core::Relationship PutS3Object::Failure("failure", "FlowFiles are routed to failure relationship");
+namespace org::apache::nifi::minifi::aws::processors {
 
 void PutS3Object::initialize() {
-  // Add new supported properties
-  setSupportedProperties({Bucket, AccessKey, SecretKey, CredentialsFile, CredentialsFile, AWSCredentialsProviderService, Region, CommunicationsTimeout,
-                          EndpointOverrideURL, ProxyHost, ProxyPort, ProxyUsername, ProxyPassword, UseDefaultCredentials, ObjectKey, ContentType, StorageClass,
-                          FullControlUserList, ReadPermissionUserList, ReadACLUserList, WriteACLUserList, CannedACL, ServerSideEncryption});
-  // Set the supported relationships
-  setSupportedRelationships({Failure, Success});
+  setSupportedProperties(properties());
+  setSupportedRelationships(relationships());
 }
 
 void PutS3Object::fillUserMetadata(const std::shared_ptr<core::ProcessContext> &context) {
@@ -292,11 +213,4 @@ void PutS3Object::onTrigger(const std::shared_ptr<core::ProcessContext> &context
   }
 }
 
-REGISTER_RESOURCE(PutS3Object, "This Processor puts FlowFiles to an Amazon S3 Bucket.");
-
-}  // namespace processors
-}  // namespace aws
-}  // namespace minifi
-}  // namespace nifi
-}  // namespace apache
-}  // namespace org
+}  // namespace org::apache::nifi::minifi::aws::processors
diff --git a/extensions/aws/processors/PutS3Object.h b/extensions/aws/processors/PutS3Object.h
index 2d1575502..ee8b89589 100644
--- a/extensions/aws/processors/PutS3Object.h
+++ b/extensions/aws/processors/PutS3Object.h
@@ -1,7 +1,4 @@
 /**
- * @file PutS3Object.h
- * PutS3Object class declaration
- *
  * 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.
@@ -32,28 +29,23 @@
 
 #include "io/StreamPipe.h"
 #include "S3Processor.h"
+#include "utils/ArrayUtils.h"
 #include "utils/gsl.h"
 #include "utils/Id.h"
 
 template<typename T>
 class S3TestsFixture;
 
-namespace org {
-namespace apache {
-namespace nifi {
-namespace minifi {
-namespace aws {
-namespace processors {
+namespace org::apache::nifi::minifi::aws::processors {
 
 class PutS3Object : public S3Processor {
  public:
-  static constexpr char const* ProcessorName = "PutS3Object";
-
   static const std::set<std::string> CANNED_ACLS;
   static const std::set<std::string> STORAGE_CLASSES;
   static const std::set<std::string> SERVER_SIDE_ENCRYPTIONS;
 
-  // Supported Properties
+  EXTENSIONAPI static constexpr const char* Description = "This Processor puts FlowFiles to an Amazon S3 Bucket.";
+
   static const core::Property ObjectKey;
   static const core::Property ContentType;
   static const core::Property StorageClass;
@@ -63,10 +55,30 @@ class PutS3Object : public S3Processor {
   static const core::Property ReadACLUserList;
   static const core::Property WriteACLUserList;
   static const core::Property CannedACL;
+  static auto properties() {
+    return minifi::utils::array_cat(S3Processor::properties(), std::array{
+      ObjectKey,
+      ContentType,
+      StorageClass,
+      ServerSideEncryption,
+      FullControlUserList,
+      ReadPermissionUserList,
+      ReadACLUserList,
+      WriteACLUserList,
+      CannedACL
+    });
+  }
+
+  EXTENSIONAPI static const core::Relationship Success;
+  EXTENSIONAPI static const core::Relationship Failure;
+  static auto relationships() { return std::array{Success, Failure}; }
 
-  // Supported Relationships
-  static const core::Relationship Failure;
-  static const core::Relationship Success;
+  EXTENSIONAPI static constexpr bool SupportsDynamicProperties = true;
+  EXTENSIONAPI static constexpr bool SupportsDynamicRelationships = false;
+  EXTENSIONAPI static constexpr core::annotation::Input InputRequirement = core::annotation::Input::INPUT_REQUIRED;
+  EXTENSIONAPI static constexpr bool IsSingleThreaded = true;
+
+  ADD_COMMON_VIRTUAL_FUNCTIONS_FOR_PROCESSORS
 
   explicit PutS3Object(const std::string& name, const minifi::utils::Identifier& uuid = minifi::utils::Identifier())
     : S3Processor(name, uuid, core::logging::LoggerFactory<PutS3Object>::getLogger()) {
@@ -80,8 +92,8 @@ class PutS3Object : public S3Processor {
 
   class ReadCallback {
    public:
-    static const uint64_t MAX_SIZE;
-    static const uint64_t BUFFER_SIZE;
+    static constexpr uint64_t MAX_SIZE = 5_GiB;
+    static constexpr uint64_t BUFFER_SIZE = 4_KiB;
 
     ReadCallback(uint64_t flow_size, const minifi::aws::s3::PutObjectRequestParameters& options, aws::s3::S3Wrapper& s3_wrapper)
       : flow_size_(flow_size)
@@ -122,14 +134,6 @@ class PutS3Object : public S3Processor {
   };
 
  private:
-  core::annotation::Input getInputRequirement() const override {
-    return core::annotation::Input::INPUT_REQUIRED;
-  }
-
-  bool isSingleThreaded() const override {
-    return true;
-  }
-
   friend class ::S3TestsFixture<PutS3Object>;
 
   explicit PutS3Object(const std::string& name, const minifi::utils::Identifier& uuid, std::unique_ptr<aws::s3::S3RequestSender> s3_request_sender)
@@ -156,9 +160,4 @@ class PutS3Object : public S3Processor {
   std::string server_side_encryption_;
 };
 
-}  // namespace processors
-}  // namespace aws
-}  // namespace minifi
-}  // namespace nifi
-}  // namespace apache
-}  // namespace org
+}  // namespace org::apache::nifi::minifi::aws::processors
diff --git a/extensions/aws/processors/S3Processor.cpp b/extensions/aws/processors/S3Processor.cpp
index 3b22ef2f7..e1b9813ac 100644
--- a/extensions/aws/processors/S3Processor.cpp
+++ b/extensions/aws/processors/S3Processor.cpp
@@ -1,7 +1,4 @@
 /**
- * @file S3Processor.cpp
- * Base S3 processor class implementation
- *
  * 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.
@@ -31,90 +28,7 @@
 #include "properties/Properties.h"
 #include "utils/StringUtils.h"
 
-namespace org {
-namespace apache {
-namespace nifi {
-namespace minifi {
-namespace aws {
-namespace processors {
-
-const std::set<std::string> S3Processor::REGIONS({region::AF_SOUTH_1, region::AP_EAST_1, region::AP_NORTHEAST_1,
-  region::AP_NORTHEAST_2, region::AP_NORTHEAST_3, region::AP_SOUTH_1, region::AP_SOUTHEAST_1, region::AP_SOUTHEAST_2,
-  region::CA_CENTRAL_1, region::CN_NORTH_1, region::CN_NORTHWEST_1, region::EU_CENTRAL_1, region::EU_NORTH_1,
-  region::EU_SOUTH_1, region::EU_WEST_1, region::EU_WEST_2, region::EU_WEST_3, region::ME_SOUTH_1, region::SA_EAST_1,
-  region::US_EAST_1, region::US_EAST_2, region::US_GOV_EAST_1, region::US_GOV_WEST_1, region::US_WEST_1, region::US_WEST_2});
-
-const core::Property S3Processor::Bucket(
-  core::PropertyBuilder::createProperty("Bucket")
-    ->withDescription("The S3 bucket")
-    ->isRequired(true)
-    ->supportsExpressionLanguage(true)
-    ->build());
-const core::Property S3Processor::AccessKey(
-  core::PropertyBuilder::createProperty("Access Key")
-    ->withDescription("AWS account access key")
-    ->supportsExpressionLanguage(true)
-    ->build());
-const core::Property S3Processor::SecretKey(
-  core::PropertyBuilder::createProperty("Secret Key")
-    ->withDescription("AWS account secret key")
-    ->supportsExpressionLanguage(true)
-    ->build());
-const core::Property S3Processor::CredentialsFile(
-  core::PropertyBuilder::createProperty("Credentials File")
-    ->withDescription("Path to a file containing AWS access key and secret key in properties file format. Properties used: accessKey and secretKey")
-    ->build());
-const core::Property S3Processor::AWSCredentialsProviderService(
-  core::PropertyBuilder::createProperty("AWS Credentials Provider service")
-    ->withDescription("The name of the AWS Credentials Provider controller service that is used to obtain AWS credentials.")
-    ->build());
-const core::Property S3Processor::Region(
-  core::PropertyBuilder::createProperty("Region")
-    ->isRequired(true)
-    ->withDefaultValue<std::string>(region::US_WEST_2)
-    ->withAllowableValues<std::string>(S3Processor::REGIONS)
-    ->withDescription("AWS Region")
-    ->build());
-const core::Property S3Processor::CommunicationsTimeout(
-  core::PropertyBuilder::createProperty("Communications Timeout")
-    ->isRequired(true)
-    ->withDefaultValue<core::TimePeriodValue>("30 sec")
-    ->withDescription("Sets the timeout of the communication between the AWS server and the client")
-    ->build());
-const core::Property S3Processor::EndpointOverrideURL(
-  core::PropertyBuilder::createProperty("Endpoint Override URL")
-    ->withDescription("Endpoint URL to use instead of the AWS default including scheme, host, "
-                      "port, and path. The AWS libraries select an endpoint URL based on the AWS "
-                      "region, but this property overrides the selected endpoint URL, allowing use "
-                      "with other S3-compatible endpoints.")
-    ->supportsExpressionLanguage(true)
-    ->build());
-const core::Property S3Processor::ProxyHost(
-  core::PropertyBuilder::createProperty("Proxy Host")
-    ->withDescription("Proxy host name or IP")
-    ->supportsExpressionLanguage(true)
-    ->build());
-const core::Property S3Processor::ProxyPort(
-  core::PropertyBuilder::createProperty("Proxy Port")
-    ->withDescription("The port number of the proxy host")
-    ->supportsExpressionLanguage(true)
-    ->build());
-const core::Property S3Processor::ProxyUsername(
-    core::PropertyBuilder::createProperty("Proxy Username")
-    ->withDescription("Username to set when authenticating against proxy")
-    ->supportsExpressionLanguage(true)
-    ->build());
-const core::Property S3Processor::ProxyPassword(
-  core::PropertyBuilder::createProperty("Proxy Password")
-    ->withDescription("Password to set when authenticating against proxy")
-    ->supportsExpressionLanguage(true)
-    ->build());
-const core::Property S3Processor::UseDefaultCredentials(
-    core::PropertyBuilder::createProperty("Use Default Credentials")
-    ->withDescription("If true, uses the Default Credential chain, including EC2 instance profiles or roles, environment variables, default user credentials, etc.")
-    ->withDefaultValue<bool>(false)
-    ->isRequired(true)
-    ->build());
+namespace org::apache::nifi::minifi::aws::processors {
 
 S3Processor::S3Processor(const std::string& name, const minifi::utils::Identifier& uuid, std::shared_ptr<core::logging::Logger> logger)
   : core::Processor(name, uuid),
@@ -243,9 +157,4 @@ std::optional<CommonProperties> S3Processor::getCommonELSupportedProperties(
   return properties;
 }
 
-}  // namespace processors
-}  // namespace aws
-}  // namespace minifi
-}  // namespace nifi
-}  // namespace apache
-}  // namespace org
+}  // namespace org::apache::nifi::minifi::aws::processors
diff --git a/extensions/aws/processors/S3Processor.h b/extensions/aws/processors/S3Processor.h
index 2957d0d0c..10cfc50d1 100644
--- a/extensions/aws/processors/S3Processor.h
+++ b/extensions/aws/processors/S3Processor.h
@@ -85,24 +85,39 @@ class S3Processor : public core::Processor {
  public:
   static const std::set<std::string> REGIONS;
 
-  // Supported Properties
-  static const core::Property Bucket;
-  static const core::Property AccessKey;
-  static const core::Property SecretKey;
-  static const core::Property CredentialsFile;
-  static const core::Property AWSCredentialsProviderService;
-  static const core::Property Region;
-  static const core::Property CommunicationsTimeout;
-  static const core::Property EndpointOverrideURL;
-  static const core::Property ProxyHost;
-  static const core::Property ProxyPort;
-  static const core::Property ProxyUsername;
-  static const core::Property ProxyPassword;
-  static const core::Property UseDefaultCredentials;
+  EXTENSIONAPI static const core::Property Bucket;
+  EXTENSIONAPI static const core::Property AccessKey;
+  EXTENSIONAPI static const core::Property SecretKey;
+  EXTENSIONAPI static const core::Property CredentialsFile;
+  EXTENSIONAPI static const core::Property AWSCredentialsProviderService;
+  EXTENSIONAPI static const core::Property Region;
+  EXTENSIONAPI static const core::Property CommunicationsTimeout;
+  EXTENSIONAPI static const core::Property EndpointOverrideURL;
+  EXTENSIONAPI static const core::Property ProxyHost;
+  EXTENSIONAPI static const core::Property ProxyPort;
+  EXTENSIONAPI static const core::Property ProxyUsername;
+  EXTENSIONAPI static const core::Property ProxyPassword;
+  EXTENSIONAPI static const core::Property UseDefaultCredentials;
+  static auto properties() {
+    return std::array{
+      Bucket,
+      AccessKey,
+      SecretKey,
+      CredentialsFile,
+      AWSCredentialsProviderService,
+      Region,
+      CommunicationsTimeout,
+      EndpointOverrideURL,
+      ProxyHost,
+      ProxyPort,
+      ProxyUsername,
+      ProxyPassword,
+      UseDefaultCredentials
+    };
+  }
 
   explicit S3Processor(const std::string& name, const minifi::utils::Identifier& uuid, std::shared_ptr<core::logging::Logger> logger);
 
-  bool supportsDynamicProperties() override { return true; }
   void onSchedule(const std::shared_ptr<core::ProcessContext> &context, const std::shared_ptr<core::ProcessSessionFactory> &sessionFactory) override;
 
  protected:
diff --git a/extensions/aws/processors/S3ProcessorStaticDefinitions.cpp b/extensions/aws/processors/S3ProcessorStaticDefinitions.cpp
new file mode 100644
index 000000000..1bc96788f
--- /dev/null
+++ b/extensions/aws/processors/S3ProcessorStaticDefinitions.cpp
@@ -0,0 +1,273 @@
+/**
+ * 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 "DeleteS3Object.h"
+#include "FetchS3Object.h"
+#include "ListS3.h"
+#include "PutS3Object.h"
+#include "S3Processor.h"
+#include "core/PropertyBuilder.h"
+#include "core/Resource.h"
+#include "utils/MapUtils.h"
+
+// FIXME(fgerlits): we need to put all these static definitions in a single file so that they are executed in this order at runtime
+// when https://issues.apache.org/jira/browse/MINIFICPP-1825 is closed, these definitions should be moved back to the cpp file of the class to which they belong
+
+namespace org::apache::nifi::minifi::aws::processors {
+
+// S3Processor
+
+const std::set<std::string> S3Processor::REGIONS({region::AF_SOUTH_1, region::AP_EAST_1, region::AP_NORTHEAST_1,
+  region::AP_NORTHEAST_2, region::AP_NORTHEAST_3, region::AP_SOUTH_1, region::AP_SOUTHEAST_1, region::AP_SOUTHEAST_2,
+  region::CA_CENTRAL_1, region::CN_NORTH_1, region::CN_NORTHWEST_1, region::EU_CENTRAL_1, region::EU_NORTH_1,
+  region::EU_SOUTH_1, region::EU_WEST_1, region::EU_WEST_2, region::EU_WEST_3, region::ME_SOUTH_1, region::SA_EAST_1,
+  region::US_EAST_1, region::US_EAST_2, region::US_GOV_EAST_1, region::US_GOV_WEST_1, region::US_WEST_1, region::US_WEST_2});
+
+const core::Property S3Processor::Bucket(
+  core::PropertyBuilder::createProperty("Bucket")
+    ->withDescription("The S3 bucket")
+    ->isRequired(true)
+    ->supportsExpressionLanguage(true)
+    ->build());
+const core::Property S3Processor::AccessKey(
+  core::PropertyBuilder::createProperty("Access Key")
+    ->withDescription("AWS account access key")
+    ->supportsExpressionLanguage(true)
+    ->build());
+const core::Property S3Processor::SecretKey(
+  core::PropertyBuilder::createProperty("Secret Key")
+    ->withDescription("AWS account secret key")
+    ->supportsExpressionLanguage(true)
+    ->build());
+const core::Property S3Processor::CredentialsFile(
+  core::PropertyBuilder::createProperty("Credentials File")
+    ->withDescription("Path to a file containing AWS access key and secret key in properties file format. Properties used: accessKey and secretKey")
+    ->build());
+const core::Property S3Processor::AWSCredentialsProviderService(
+  core::PropertyBuilder::createProperty("AWS Credentials Provider service")
+    ->withDescription("The name of the AWS Credentials Provider controller service that is used to obtain AWS credentials.")
+    ->build());
+const core::Property S3Processor::Region(
+  core::PropertyBuilder::createProperty("Region")
+    ->isRequired(true)
+    ->withDefaultValue<std::string>(region::US_WEST_2)
+    ->withAllowableValues<std::string>(S3Processor::REGIONS)
+    ->withDescription("AWS Region")
+    ->build());
+const core::Property S3Processor::CommunicationsTimeout(
+  core::PropertyBuilder::createProperty("Communications Timeout")
+    ->isRequired(true)
+    ->withDefaultValue<core::TimePeriodValue>("30 sec")
+    ->withDescription("Sets the timeout of the communication between the AWS server and the client")
+    ->build());
+const core::Property S3Processor::EndpointOverrideURL(
+  core::PropertyBuilder::createProperty("Endpoint Override URL")
+    ->withDescription("Endpoint URL to use instead of the AWS default including scheme, host, "
+                      "port, and path. The AWS libraries select an endpoint URL based on the AWS "
+                      "region, but this property overrides the selected endpoint URL, allowing use "
+                      "with other S3-compatible endpoints.")
+    ->supportsExpressionLanguage(true)
+    ->build());
+const core::Property S3Processor::ProxyHost(
+  core::PropertyBuilder::createProperty("Proxy Host")
+    ->withDescription("Proxy host name or IP")
+    ->supportsExpressionLanguage(true)
+    ->build());
+const core::Property S3Processor::ProxyPort(
+  core::PropertyBuilder::createProperty("Proxy Port")
+    ->withDescription("The port number of the proxy host")
+    ->supportsExpressionLanguage(true)
+    ->build());
+const core::Property S3Processor::ProxyUsername(
+    core::PropertyBuilder::createProperty("Proxy Username")
+    ->withDescription("Username to set when authenticating against proxy")
+    ->supportsExpressionLanguage(true)
+    ->build());
+const core::Property S3Processor::ProxyPassword(
+  core::PropertyBuilder::createProperty("Proxy Password")
+    ->withDescription("Password to set when authenticating against proxy")
+    ->supportsExpressionLanguage(true)
+    ->build());
+const core::Property S3Processor::UseDefaultCredentials(
+    core::PropertyBuilder::createProperty("Use Default Credentials")
+    ->withDescription("If true, uses the Default Credential chain, including EC2 instance profiles or roles, environment variables, default user credentials, etc.")
+    ->withDefaultValue<bool>(false)
+    ->isRequired(true)
+    ->build());
+
+
+// DeleteS3Object
+
+const core::Property DeleteS3Object::ObjectKey(
+  core::PropertyBuilder::createProperty("Object Key")
+    ->withDescription("The key of the S3 object. If none is given the filename attribute will be used by default.")
+    ->supportsExpressionLanguage(true)
+    ->build());
+const core::Property DeleteS3Object::Version(
+  core::PropertyBuilder::createProperty("Version")
+    ->withDescription("The Version of the Object to delete")
+    ->supportsExpressionLanguage(true)
+    ->build());
+
+const core::Relationship DeleteS3Object::Success("success", "FlowFiles are routed to success relationship");
+const core::Relationship DeleteS3Object::Failure("failure", "FlowFiles are routed to failure relationship");
+
+REGISTER_RESOURCE(DeleteS3Object, Processor);
+
+
+// FetchS3Object
+
+const core::Property FetchS3Object::ObjectKey(
+  core::PropertyBuilder::createProperty("Object Key")
+    ->withDescription("The key of the S3 object. If none is given the filename attribute will be used by default.")
+    ->supportsExpressionLanguage(true)
+    ->build());
+const core::Property FetchS3Object::Version(
+  core::PropertyBuilder::createProperty("Version")
+    ->withDescription("The Version of the Object to download")
+    ->supportsExpressionLanguage(true)
+    ->build());
+const core::Property FetchS3Object::RequesterPays(
+  core::PropertyBuilder::createProperty("Requester Pays")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(false)
+    ->withDescription("If true, indicates that the requester consents to pay any charges associated with retrieving "
+                      "objects from the S3 bucket. This sets the 'x-amz-request-payer' header to 'requester'.")
+    ->build());
+
+const core::Relationship FetchS3Object::Success("success", "FlowFiles are routed to success relationship");
+const core::Relationship FetchS3Object::Failure("failure", "FlowFiles are routed to failure relationship");
+
+REGISTER_RESOURCE(FetchS3Object, Processor);
+
+
+// ListS3
+
+const core::Property ListS3::Delimiter(
+  core::PropertyBuilder::createProperty("Delimiter")
+    ->withDescription("The string used to delimit directories within the bucket. Please consult the AWS documentation for the correct use of this field.")
+    ->build());
+const core::Property ListS3::Prefix(
+  core::PropertyBuilder::createProperty("Prefix")
+    ->withDescription("The prefix used to filter the object list. In most cases, it should end with a forward slash ('/').")
+    ->build());
+const core::Property ListS3::UseVersions(
+  core::PropertyBuilder::createProperty("Use Versions")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(false)
+    ->withDescription("Specifies whether to use S3 versions, if applicable. If false, only the latest version of each object will be returned.")
+    ->build());
+const core::Property ListS3::MinimumObjectAge(
+  core::PropertyBuilder::createProperty("Minimum Object Age")
+    ->isRequired(true)
+    ->withDefaultValue<core::TimePeriodValue>("0 sec")
+    ->withDescription("The minimum age that an S3 object must be in order to be considered; any object younger than this amount of time (according to last modification date) will be ignored.")
+    ->build());
+const core::Property ListS3::WriteObjectTags(
+  core::PropertyBuilder::createProperty("Write Object Tags")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(false)
+    ->withDescription("If set to 'true', the tags associated with the S3 object will be written as FlowFile attributes.")
+    ->build());
+const core::Property ListS3::WriteUserMetadata(
+  core::PropertyBuilder::createProperty("Write User Metadata")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(false)
+    ->withDescription("If set to 'true', the user defined metadata associated with the S3 object will be added to FlowFile attributes/records.")
+    ->build());
+const core::Property ListS3::RequesterPays(
+  core::PropertyBuilder::createProperty("Requester Pays")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(false)
+    ->withDescription("If true, indicates that the requester consents to pay any charges associated with listing the S3 bucket. This sets the 'x-amz-request-payer' header to 'requester'. "
+                      "Note that this setting is only used if Write User Metadata is true.")
+    ->build());
+
+const core::Relationship ListS3::Success("success", "FlowFiles are routed to success relationship");
+
+REGISTER_RESOURCE(ListS3, Processor);
+
+
+// PutS3Object
+
+const std::set<std::string> PutS3Object::CANNED_ACLS(minifi::utils::MapUtils::getKeys(minifi::aws::s3::CANNED_ACL_MAP));
+const std::set<std::string> PutS3Object::STORAGE_CLASSES(minifi::utils::MapUtils::getKeys(minifi::aws::s3::STORAGE_CLASS_MAP));
+const std::set<std::string> PutS3Object::SERVER_SIDE_ENCRYPTIONS(minifi::utils::MapUtils::getKeys(minifi::aws::s3::SERVER_SIDE_ENCRYPTION_MAP));
+
+const core::Property PutS3Object::ObjectKey(
+  core::PropertyBuilder::createProperty("Object Key")
+    ->withDescription("The key of the S3 object. If none is given the filename attribute will be used by default.")
+    ->supportsExpressionLanguage(true)
+    ->build());
+const core::Property PutS3Object::ContentType(
+  core::PropertyBuilder::createProperty("Content Type")
+    ->withDescription("Sets the Content-Type HTTP header indicating the type of content stored in "
+                      "the associated object. The value of this header is a standard MIME type. "
+                      "If no content type is provided the default content type "
+                      "\"application/octet-stream\" will be used.")
+    ->supportsExpressionLanguage(true)
+    ->withDefaultValue<std::string>("application/octet-stream")
+    ->build());
+const core::Property PutS3Object::StorageClass(
+  core::PropertyBuilder::createProperty("Storage Class")
+    ->isRequired(true)
+    ->withDefaultValue<std::string>("Standard")
+    ->withAllowableValues<std::string>(PutS3Object::STORAGE_CLASSES)
+    ->withDescription("AWS S3 Storage Class")
+    ->build());
+const core::Property PutS3Object::FullControlUserList(
+  core::PropertyBuilder::createProperty("FullControl User List")
+    ->withDescription("A comma-separated list of Amazon User ID's or E-mail addresses that specifies who should have Full Control for an object.")
+    ->supportsExpressionLanguage(true)
+    ->build());
+const core::Property PutS3Object::ReadPermissionUserList(
+  core::PropertyBuilder::createProperty("Read Permission User List")
+    ->withDescription("A comma-separated list of Amazon User ID's or E-mail addresses that specifies who should have Read Access for an object.")
+    ->supportsExpressionLanguage(true)
+    ->build());
+const core::Property PutS3Object::ReadACLUserList(
+  core::PropertyBuilder::createProperty("Read ACL User List")
+    ->withDescription("A comma-separated list of Amazon User ID's or E-mail addresses that specifies who should have permissions to read "
+                      "the Access Control List for an object.")
+    ->supportsExpressionLanguage(true)
+    ->build());
+const core::Property PutS3Object::WriteACLUserList(
+  core::PropertyBuilder::createProperty("Write ACL User List")
+    ->withDescription("A comma-separated list of Amazon User ID's or E-mail addresses that specifies who should have permissions to change "
+                      "the Access Control List for an object.")
+    ->supportsExpressionLanguage(true)
+    ->build());
+const core::Property PutS3Object::CannedACL(
+  core::PropertyBuilder::createProperty("Canned ACL")
+    ->withDescription("Amazon Canned ACL for an object. Allowed values: BucketOwnerFullControl, BucketOwnerRead, AuthenticatedRead, "
+                      "PublicReadWrite, PublicRead, Private, AwsExecRead; will be ignored if any other ACL/permission property is specified.")
+    ->supportsExpressionLanguage(true)
+    ->build());
+const core::Property PutS3Object::ServerSideEncryption(
+  core::PropertyBuilder::createProperty("Server Side Encryption")
+    ->isRequired(true)
+    ->withDefaultValue<std::string>("None")
+    ->withAllowableValues<std::string>(PutS3Object::SERVER_SIDE_ENCRYPTIONS)
+    ->withDescription("Specifies the algorithm used for server side encryption.")
+    ->build());
+
+const core::Relationship PutS3Object::Success("success", "FlowFiles are routed to success relationship");
+const core::Relationship PutS3Object::Failure("failure", "FlowFiles are routed to failure relationship");
+
+REGISTER_RESOURCE(PutS3Object, Processor);
+
+}  // namespace org::apache::nifi::minifi::aws::processors
diff --git a/extensions/azure/controllerservices/AzureStorageCredentialsService.cpp b/extensions/azure/controllerservices/AzureStorageCredentialsService.cpp
index 0acebea92..951bf7ce2 100644
--- a/extensions/azure/controllerservices/AzureStorageCredentialsService.cpp
+++ b/extensions/azure/controllerservices/AzureStorageCredentialsService.cpp
@@ -20,6 +20,7 @@
 
 #include <set>
 
+#include "core/PropertyBuilder.h"
 #include "core/Resource.h"
 
 namespace org::apache::nifi::minifi::azure::controllers {
@@ -54,7 +55,7 @@ const core::Property AzureStorageCredentialsService::UseManagedIdentityCredentia
     ->build());
 
 void AzureStorageCredentialsService::initialize() {
-  setSupportedProperties({StorageAccountName, StorageAccountKey, SASToken, CommonStorageAccountEndpointSuffix, ConnectionString, UseManagedIdentityCredentials});
+  setSupportedProperties(properties());
 }
 
 void AzureStorageCredentialsService::onEnable() {
@@ -80,6 +81,6 @@ void AzureStorageCredentialsService::onEnable() {
   }
 }
 
-REGISTER_RESOURCE(AzureStorageCredentialsService, "Azure Storage Credentials Management Service");
+REGISTER_RESOURCE(AzureStorageCredentialsService, ControllerService);
 
 }  // namespace org::apache::nifi::minifi::azure::controllers
diff --git a/extensions/azure/controllerservices/AzureStorageCredentialsService.h b/extensions/azure/controllerservices/AzureStorageCredentialsService.h
index 14928e594..ecc641636 100644
--- a/extensions/azure/controllerservices/AzureStorageCredentialsService.h
+++ b/extensions/azure/controllerservices/AzureStorageCredentialsService.h
@@ -30,12 +30,27 @@ namespace org::apache::nifi::minifi::azure::controllers {
 
 class AzureStorageCredentialsService : public core::controller::ControllerService {
  public:
+  EXTENSIONAPI static constexpr const char* Description = "Azure Storage Credentials Management Service";
+
   EXTENSIONAPI static const core::Property StorageAccountName;
   EXTENSIONAPI static const core::Property StorageAccountKey;
   EXTENSIONAPI static const core::Property SASToken;
   EXTENSIONAPI static const core::Property CommonStorageAccountEndpointSuffix;
   EXTENSIONAPI static const core::Property ConnectionString;
   EXTENSIONAPI static const core::Property UseManagedIdentityCredentials;
+  static auto properties() {
+    return std::array{
+      StorageAccountName,
+      StorageAccountKey,
+      SASToken,
+      CommonStorageAccountEndpointSuffix,
+      ConnectionString,
+      UseManagedIdentityCredentials
+    };
+  }
+
+  EXTENSIONAPI static constexpr bool SupportsDynamicProperties = false;
+  ADD_COMMON_VIRTUAL_FUNCTIONS_FOR_CONTROLLER_SERVICES
 
   explicit AzureStorageCredentialsService(const std::string& name, const minifi::utils::Identifier& uuid = {})
       : ControllerService(name, uuid) {
diff --git a/extensions/azure/processors/AzureBlobStorageProcessorBase.cpp b/extensions/azure/processors/AzureBlobStorageProcessorBase.cpp
index 28096e989..039d20c3b 100644
--- a/extensions/azure/processors/AzureBlobStorageProcessorBase.cpp
+++ b/extensions/azure/processors/AzureBlobStorageProcessorBase.cpp
@@ -24,46 +24,6 @@
 
 namespace org::apache::nifi::minifi::azure::processors {
 
-const core::Property AzureBlobStorageProcessorBase::ContainerName(
-  core::PropertyBuilder::createProperty("Container Name")
-    ->withDescription("Name of the Azure Storage container. In case of PutAzureBlobStorage processor, container can be created if it does not exist.")
-    ->supportsExpressionLanguage(true)
-    ->isRequired(true)
-    ->build());
-const core::Property AzureBlobStorageProcessorBase::StorageAccountName(
-    core::PropertyBuilder::createProperty("Storage Account Name")
-      ->withDescription("The storage account name.")
-      ->supportsExpressionLanguage(true)
-      ->build());
-const core::Property AzureBlobStorageProcessorBase::StorageAccountKey(
-    core::PropertyBuilder::createProperty("Storage Account Key")
-      ->withDescription("The storage account key. This is an admin-like password providing access to every container in this account. "
-                        "It is recommended one uses Shared Access Signature (SAS) token instead for fine-grained control with policies.")
-      ->supportsExpressionLanguage(true)
-      ->build());
-const core::Property AzureBlobStorageProcessorBase::SASToken(
-    core::PropertyBuilder::createProperty("SAS Token")
-      ->withDescription("Shared Access Signature token. Specify either SAS Token (recommended) or Storage Account Key together with Storage Account Name if Managed Identity is not used.")
-      ->supportsExpressionLanguage(true)
-      ->build());
-const core::Property AzureBlobStorageProcessorBase::CommonStorageAccountEndpointSuffix(
-    core::PropertyBuilder::createProperty("Common Storage Account Endpoint Suffix")
-      ->withDescription("Storage accounts in public Azure always use a common FQDN suffix. Override this endpoint suffix with a "
-                        "different suffix in certain circumstances (like Azure Stack or non-public Azure regions). ")
-      ->supportsExpressionLanguage(true)
-      ->build());
-const core::Property AzureBlobStorageProcessorBase::ConnectionString(
-  core::PropertyBuilder::createProperty("Connection String")
-    ->withDescription("Connection string used to connect to Azure Storage service. This overrides all other set credential properties if Managed Identity is not used.")
-    ->supportsExpressionLanguage(true)
-    ->build());
-const core::Property AzureBlobStorageProcessorBase::UseManagedIdentityCredentials(
-  core::PropertyBuilder::createProperty("Use Managed Identity Credentials")
-    ->withDescription("If true Managed Identity credentials will be used together with the Storage Account Name for authentication.")
-    ->isRequired(true)
-    ->withDefaultValue<bool>(false)
-    ->build());
-
 void AzureBlobStorageProcessorBase::onSchedule(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSessionFactory>& /*sessionFactory*/) {
   gsl_Expects(context);
   std::string value;
diff --git a/extensions/azure/processors/AzureBlobStorageProcessorBase.h b/extensions/azure/processors/AzureBlobStorageProcessorBase.h
index 3aea33c3c..b4970d481 100644
--- a/extensions/azure/processors/AzureBlobStorageProcessorBase.h
+++ b/extensions/azure/processors/AzureBlobStorageProcessorBase.h
@@ -30,12 +30,12 @@
 #include "storage/AzureBlobStorage.h"
 #include "AzureStorageProcessorBase.h"
 #include "storage/AzureStorageCredentials.h"
+#include "utils/ArrayUtils.h"
 
 namespace org::apache::nifi::minifi::azure::processors {
 
 class AzureBlobStorageProcessorBase : public AzureStorageProcessorBase {
  public:
-  // Supported Properties
   EXTENSIONAPI static const core::Property ContainerName;
   EXTENSIONAPI static const core::Property StorageAccountName;
   EXTENSIONAPI static const core::Property StorageAccountKey;
@@ -43,6 +43,17 @@ class AzureBlobStorageProcessorBase : public AzureStorageProcessorBase {
   EXTENSIONAPI static const core::Property CommonStorageAccountEndpointSuffix;
   EXTENSIONAPI static const core::Property ConnectionString;
   EXTENSIONAPI static const core::Property UseManagedIdentityCredentials;
+  static auto properties() {
+    return utils::array_cat(AzureStorageProcessorBase::properties(), std::array{
+      ContainerName,
+      StorageAccountName,
+      StorageAccountKey,
+      SASToken,
+      CommonStorageAccountEndpointSuffix,
+      ConnectionString,
+      UseManagedIdentityCredentials
+    });
+  }
 
   explicit AzureBlobStorageProcessorBase(const std::string& name, const minifi::utils::Identifier& uuid, const std::shared_ptr<core::logging::Logger>& logger)
     : AzureBlobStorageProcessorBase(name, uuid, logger, nullptr) {
diff --git a/extensions/azure/processors/AzureBlobStorageSingleBlobProcessorBase.cpp b/extensions/azure/processors/AzureBlobStorageSingleBlobProcessorBase.cpp
index a7413ed53..6554cf3f8 100644
--- a/extensions/azure/processors/AzureBlobStorageSingleBlobProcessorBase.cpp
+++ b/extensions/azure/processors/AzureBlobStorageSingleBlobProcessorBase.cpp
@@ -24,13 +24,6 @@
 
 namespace org::apache::nifi::minifi::azure::processors {
 
-const core::Property AzureBlobStorageSingleBlobProcessorBase::Blob(
-  core::PropertyBuilder::createProperty("Blob")
-    ->withDescription("The filename of the blob. If left empty the filename attribute will be used by default.")
-    ->supportsExpressionLanguage(true)
-    ->build());
-
-
 bool AzureBlobStorageSingleBlobProcessorBase::setBlobOperationParameters(
     storage::AzureBlobStorageBlobOperationParameters& params,
     core::ProcessContext &context,
diff --git a/extensions/azure/processors/AzureBlobStorageSingleBlobProcessorBase.h b/extensions/azure/processors/AzureBlobStorageSingleBlobProcessorBase.h
index 1b948e75f..ed5d8f4c0 100644
--- a/extensions/azure/processors/AzureBlobStorageSingleBlobProcessorBase.h
+++ b/extensions/azure/processors/AzureBlobStorageSingleBlobProcessorBase.h
@@ -26,13 +26,16 @@
 #include <utility>
 
 #include "AzureBlobStorageProcessorBase.h"
+#include "utils/ArrayUtils.h"
 
 namespace org::apache::nifi::minifi::azure::processors {
 
 class AzureBlobStorageSingleBlobProcessorBase : public AzureBlobStorageProcessorBase {
  public:
-  // Supported Properties
   EXTENSIONAPI static const core::Property Blob;
+  static auto properties() {
+    return utils::array_cat(AzureBlobStorageProcessorBase::properties(), std::array{Blob});
+  }
 
   explicit AzureBlobStorageSingleBlobProcessorBase(const std::string& name, const minifi::utils::Identifier& uuid, const std::shared_ptr<core::logging::Logger>& logger)
     : AzureBlobStorageSingleBlobProcessorBase(name, uuid, logger, nullptr) {
diff --git a/extensions/azure/processors/AzureDataLakeStorageFileProcessorBase.cpp b/extensions/azure/processors/AzureDataLakeStorageFileProcessorBase.cpp
index 1ddee04c5..299d6b933 100644
--- a/extensions/azure/processors/AzureDataLakeStorageFileProcessorBase.cpp
+++ b/extensions/azure/processors/AzureDataLakeStorageFileProcessorBase.cpp
@@ -25,12 +25,6 @@
 
 namespace org::apache::nifi::minifi::azure::processors {
 
-const core::Property AzureDataLakeStorageFileProcessorBase::FileName(
-    core::PropertyBuilder::createProperty("File Name")
-      ->withDescription("The filename in Azure Storage. If left empty the filename attribute will be used by default.")
-      ->supportsExpressionLanguage(true)
-      ->build());
-
 bool AzureDataLakeStorageFileProcessorBase::setFileOperationCommonParameters(
     storage::AzureDataLakeStorageFileOperationParameters& params, core::ProcessContext& context, const std::shared_ptr<core::FlowFile>& flow_file) {
   if (!setCommonParameters(params, context, flow_file)) {
diff --git a/extensions/azure/processors/AzureDataLakeStorageFileProcessorBase.h b/extensions/azure/processors/AzureDataLakeStorageFileProcessorBase.h
index 0f9a00305..17bc17863 100644
--- a/extensions/azure/processors/AzureDataLakeStorageFileProcessorBase.h
+++ b/extensions/azure/processors/AzureDataLakeStorageFileProcessorBase.h
@@ -1,7 +1,4 @@
 /**
- * @file AzureDataLakeStorageFileProcessorBase.h
- * AzureDataLakeStorageFileProcessorBase class declaration
- *
  * 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.
@@ -25,13 +22,16 @@
 #include <memory>
 
 #include "AzureDataLakeStorageProcessorBase.h"
+#include "utils/ArrayUtils.h"
 
 namespace org::apache::nifi::minifi::azure::processors {
 
 class AzureDataLakeStorageFileProcessorBase : public AzureDataLakeStorageProcessorBase {
  public:
-  // Supported Properties
   EXTENSIONAPI static const core::Property FileName;
+  static auto properties() {
+    return utils::array_cat(AzureDataLakeStorageProcessorBase::properties(), std::array{FileName});
+  }
 
   explicit AzureDataLakeStorageFileProcessorBase(const std::string& name, const minifi::utils::Identifier& uuid, const std::shared_ptr<core::logging::Logger> &logger)
     : AzureDataLakeStorageProcessorBase(name, uuid, logger) {
diff --git a/extensions/azure/processors/AzureDataLakeStorageProcessorBase.cpp b/extensions/azure/processors/AzureDataLakeStorageProcessorBase.cpp
index ed526f35b..45b2de92a 100644
--- a/extensions/azure/processors/AzureDataLakeStorageProcessorBase.cpp
+++ b/extensions/azure/processors/AzureDataLakeStorageProcessorBase.cpp
@@ -25,19 +25,6 @@
 
 namespace org::apache::nifi::minifi::azure::processors {
 
-const core::Property AzureDataLakeStorageProcessorBase::FilesystemName(
-    core::PropertyBuilder::createProperty("Filesystem Name")
-      ->withDescription("Name of the Azure Storage File System. It is assumed to be already existing.")
-      ->supportsExpressionLanguage(true)
-      ->isRequired(true)
-      ->build());
-const core::Property AzureDataLakeStorageProcessorBase::DirectoryName(
-    core::PropertyBuilder::createProperty("Directory Name")
-      ->withDescription("Name of the Azure Storage Directory. The Directory Name cannot contain a leading '/'. "
-                        "If left empty it designates the root directory. The directory will be created if not already existing.")
-      ->supportsExpressionLanguage(true)
-      ->build());
-
 void AzureDataLakeStorageProcessorBase::onSchedule(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSessionFactory>& /*sessionFactory*/) {
   gsl_Expects(context);
   std::optional<storage::AzureStorageCredentials> credentials;
diff --git a/extensions/azure/processors/AzureDataLakeStorageProcessorBase.h b/extensions/azure/processors/AzureDataLakeStorageProcessorBase.h
index 65c3232dd..298666fa6 100644
--- a/extensions/azure/processors/AzureDataLakeStorageProcessorBase.h
+++ b/extensions/azure/processors/AzureDataLakeStorageProcessorBase.h
@@ -29,15 +29,21 @@
 #include "core/logging/Logger.h"
 #include "core/logging/LoggerConfiguration.h"
 #include "storage/AzureDataLakeStorage.h"
+#include "utils/ArrayUtils.h"
 #include "AzureStorageProcessorBase.h"
 
 namespace org::apache::nifi::minifi::azure::processors {
 
 class AzureDataLakeStorageProcessorBase : public AzureStorageProcessorBase {
  public:
-  // Supported Properties
   EXTENSIONAPI static const core::Property FilesystemName;
   EXTENSIONAPI static const core::Property DirectoryName;
+  static auto properties() {
+    return utils::array_cat(AzureStorageProcessorBase::properties(), std::array{
+      FilesystemName,
+      DirectoryName
+    });
+  }
 
   explicit AzureDataLakeStorageProcessorBase(const std::string& name, const minifi::utils::Identifier& uuid, const std::shared_ptr<core::logging::Logger> &logger)
     : AzureStorageProcessorBase(name, uuid, logger) {
diff --git a/extensions/azure/processors/AzureStorageProcessorBase.cpp b/extensions/azure/processors/AzureStorageProcessorBase.cpp
index 7411617e5..fedaaa732 100644
--- a/extensions/azure/processors/AzureStorageProcessorBase.cpp
+++ b/extensions/azure/processors/AzureStorageProcessorBase.cpp
@@ -28,11 +28,6 @@
 
 namespace org::apache::nifi::minifi::azure::processors {
 
-const core::Property AzureStorageProcessorBase::AzureStorageCredentialsService(
-  core::PropertyBuilder::createProperty("Azure Storage Credentials Service")
-    ->withDescription("Name of the Azure Storage Credentials Service used to retrieve the connection string from.")
-    ->build());
-
 std::tuple<AzureStorageProcessorBase::GetCredentialsFromControllerResult, std::optional<storage::AzureStorageCredentials>> AzureStorageProcessorBase::getCredentialsFromControllerService(
     core::ProcessContext &context) const {
   std::string service_name;
diff --git a/extensions/azure/processors/AzureStorageProcessorBase.h b/extensions/azure/processors/AzureStorageProcessorBase.h
index a85ae7b63..715d5d030 100644
--- a/extensions/azure/processors/AzureStorageProcessorBase.h
+++ b/extensions/azure/processors/AzureStorageProcessorBase.h
@@ -34,8 +34,8 @@ namespace org::apache::nifi::minifi::azure::processors {
 
 class AzureStorageProcessorBase : public core::Processor {
  public:
-  // Supported Properties
   EXTENSIONAPI static const core::Property AzureStorageCredentialsService;
+  static auto properties() { return std::array{AzureStorageCredentialsService}; }
 
   AzureStorageProcessorBase(const std::string& name, const minifi::utils::Identifier& uuid, const std::shared_ptr<core::logging::Logger>& logger)
     : core::Processor(name, uuid),
diff --git a/extensions/azure/processors/AzureStorageProcessorStaticDefinitions.cpp b/extensions/azure/processors/AzureStorageProcessorStaticDefinitions.cpp
new file mode 100644
index 000000000..0205b7be6
--- /dev/null
+++ b/extensions/azure/processors/AzureStorageProcessorStaticDefinitions.cpp
@@ -0,0 +1,291 @@
+/**
+ * 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 "AzureBlobStorageProcessorBase.h"
+#include "AzureBlobStorageSingleBlobProcessorBase.h"
+#include "AzureDataLakeStorageFileProcessorBase.h"
+#include "AzureDataLakeStorageProcessorBase.h"
+#include "AzureStorageProcessorBase.h"
+#include "DeleteAzureBlobStorage.h"
+#include "DeleteAzureDataLakeStorage.h"
+#include "FetchAzureBlobStorage.h"
+#include "FetchAzureDataLakeStorage.h"
+#include "ListAzureBlobStorage.h"
+#include "ListAzureDataLakeStorage.h"
+#include "PutAzureBlobStorage.h"
+#include "PutAzureDataLakeStorage.h"
+#include "core/PropertyBuilder.h"
+#include "core/Resource.h"
+
+// FIXME(fgerlits): we need to put all these static definitions in a single file so that they are executed in this order at runtime
+// when https://issues.apache.org/jira/browse/MINIFICPP-1825 is closed, these definitions should be moved back to the cpp file of the class to which they belong
+
+namespace org::apache::nifi::minifi::azure::processors {
+
+// AzureStorageProcessorBase
+
+const core::Property AzureStorageProcessorBase::AzureStorageCredentialsService(
+  core::PropertyBuilder::createProperty("Azure Storage Credentials Service")
+    ->withDescription("Name of the Azure Storage Credentials Service used to retrieve the connection string from.")
+    ->build());
+
+
+// AzureBlobStorageProcessorBase
+
+const core::Property AzureBlobStorageProcessorBase::ContainerName(
+  core::PropertyBuilder::createProperty("Container Name")
+    ->withDescription("Name of the Azure Storage container. In case of PutAzureBlobStorage processor, container can be created if it does not exist.")
+    ->supportsExpressionLanguage(true)
+    ->isRequired(true)
+    ->build());
+
+const core::Property AzureBlobStorageProcessorBase::StorageAccountName(
+    core::PropertyBuilder::createProperty("Storage Account Name")
+      ->withDescription("The storage account name.")
+      ->supportsExpressionLanguage(true)
+      ->build());
+
+const core::Property AzureBlobStorageProcessorBase::StorageAccountKey(
+    core::PropertyBuilder::createProperty("Storage Account Key")
+      ->withDescription("The storage account key. This is an admin-like password providing access to every container in this account. "
+                        "It is recommended one uses Shared Access Signature (SAS) token instead for fine-grained control with policies.")
+      ->supportsExpressionLanguage(true)
+      ->build());
+
+const core::Property AzureBlobStorageProcessorBase::SASToken(
+    core::PropertyBuilder::createProperty("SAS Token")
+      ->withDescription("Shared Access Signature token. Specify either SAS Token (recommended) or Storage Account Key together with Storage Account Name if Managed Identity is not used.")
+      ->supportsExpressionLanguage(true)
+      ->build());
+
+const core::Property AzureBlobStorageProcessorBase::CommonStorageAccountEndpointSuffix(
+    core::PropertyBuilder::createProperty("Common Storage Account Endpoint Suffix")
+      ->withDescription("Storage accounts in public Azure always use a common FQDN suffix. Override this endpoint suffix with a "
+                        "different suffix in certain circumstances (like Azure Stack or non-public Azure regions). ")
+      ->supportsExpressionLanguage(true)
+      ->build());
+
+const core::Property AzureBlobStorageProcessorBase::ConnectionString(
+  core::PropertyBuilder::createProperty("Connection String")
+    ->withDescription("Connection string used to connect to Azure Storage service. This overrides all other set credential properties if Managed Identity is not used.")
+    ->supportsExpressionLanguage(true)
+    ->build());
+
+const core::Property AzureBlobStorageProcessorBase::UseManagedIdentityCredentials(
+  core::PropertyBuilder::createProperty("Use Managed Identity Credentials")
+    ->withDescription("If true Managed Identity credentials will be used together with the Storage Account Name for authentication.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(false)
+    ->build());
+
+
+// ListAzureBlobStorage
+
+const core::Property ListAzureBlobStorage::ListingStrategy(
+  core::PropertyBuilder::createProperty("Listing Strategy")
+    ->withDescription("Specify how to determine new/updated entities. If 'timestamps' is selected it tracks the latest timestamp of listed entity to determine new/updated entities. "
+                      "If 'none' is selected it lists an entity without any tracking, the same entity will be listed each time on executing this processor.")
+    ->isRequired(true)
+    ->withDefaultValue<std::string>(toString(EntityTracking::TIMESTAMPS))
+    ->withAllowableValues<std::string>(EntityTracking::values())
+    ->build());
+
+const core::Property ListAzureBlobStorage::Prefix(
+  core::PropertyBuilder::createProperty("Prefix")
+    ->withDescription("Search prefix for listing")
+    ->supportsExpressionLanguage(true)
+    ->build());
+
+const core::Relationship ListAzureBlobStorage::Success("success", "All FlowFiles that are received are routed to success");
+
+REGISTER_RESOURCE(ListAzureBlobStorage, Processor);
+
+
+// AzureBlobStorageSingleBlobProcessorBase
+
+const core::Property AzureBlobStorageSingleBlobProcessorBase::Blob(
+  core::PropertyBuilder::createProperty("Blob")
+    ->withDescription("The filename of the blob. If left empty the filename attribute will be used by default.")
+    ->supportsExpressionLanguage(true)
+    ->build());
+
+
+// DeleteAzureBlobStorage
+
+const core::Property DeleteAzureBlobStorage::DeleteSnapshotsOption(
+  core::PropertyBuilder::createProperty("Delete Snapshots Option")
+    ->withDescription("Specifies the snapshot deletion options to be used when deleting a blob. None: Deletes the blob only. Include Snapshots: Delete the blob and its snapshots. "
+                      "Delete Snapshots Only: Delete only the blob's snapshots.")
+    ->isRequired(true)
+    ->withDefaultValue<std::string>(toString(storage::OptionalDeletion::NONE))
+    ->withAllowableValues<std::string>(storage::OptionalDeletion::values())
+    ->build());
+
+const core::Relationship DeleteAzureBlobStorage::Success("success", "All successfully processed FlowFiles are routed to this relationship");
+const core::Relationship DeleteAzureBlobStorage::Failure("failure", "Unsuccessful operations will be transferred to the failure relationship");
+
+REGISTER_RESOURCE(DeleteAzureBlobStorage, Processor);
+
+
+// FetchAzureBlobStorage
+
+const core::Property FetchAzureBlobStorage::RangeStart(
+  core::PropertyBuilder::createProperty("Range Start")
+    ->withDescription("The byte position at which to start reading from the blob. An empty value or a value of zero will start reading at the beginning of the blob.")
+    ->supportsExpressionLanguage(true)
+    ->build());
+
+const core::Property FetchAzureBlobStorage::RangeLength(
+  core::PropertyBuilder::createProperty("Range Length")
+    ->withDescription("The number of bytes to download from the blob, starting from the Range Start. "
+                      "An empty value or a value that extends beyond the end of the blob will read to the end of the blob.")
+    ->supportsExpressionLanguage(true)
+    ->build());
+
+const core::Relationship FetchAzureBlobStorage::Success("success", "All successfully processed FlowFiles are routed to this relationship");
+const core::Relationship FetchAzureBlobStorage::Failure("failure", "Unsuccessful operations will be transferred to the failure relationship");
+
+REGISTER_RESOURCE(FetchAzureBlobStorage, Processor);
+
+
+// PutAzureBlobStorage
+
+const core::Property PutAzureBlobStorage::CreateContainer(
+  core::PropertyBuilder::createProperty("Create Container")
+    ->withDescription("Specifies whether to check if the container exists and to automatically create it if it does not. "
+                      "Permission to list containers is required. If false, this check is not made, but the Put operation will "
+                      "fail if the container does not exist.")
+    ->isRequired(true)
+    ->withDefaultValue<bool>(false)
+    ->build());
+
+const core::Relationship PutAzureBlobStorage::Success("success", "All successfully processed FlowFiles are routed to this relationship");
+const core::Relationship PutAzureBlobStorage::Failure("failure", "Unsuccessful operations will be transferred to the failure relationship");
+
+REGISTER_RESOURCE(PutAzureBlobStorage, Processor);
+
+
+// AzureDataLakeStorageProcessorBase
+
+const core::Property AzureDataLakeStorageProcessorBase::FilesystemName(
+    core::PropertyBuilder::createProperty("Filesystem Name")
+      ->withDescription("Name of the Azure Storage File System. It is assumed to be already existing.")
+      ->supportsExpressionLanguage(true)
+      ->isRequired(true)
+      ->build());
+
+const core::Property AzureDataLakeStorageProcessorBase::DirectoryName(
+    core::PropertyBuilder::createProperty("Directory Name")
+      ->withDescription("Name of the Azure Storage Directory. The Directory Name cannot contain a leading '/'. "
+                        "If left empty it designates the root directory. The directory will be created if not already existing.")
+      ->supportsExpressionLanguage(true)
+      ->build());
+
+
+// AzureDataLakeStorageFileProcessorBase
+
+const core::Property AzureDataLakeStorageFileProcessorBase::FileName(
+    core::PropertyBuilder::createProperty("File Name")
+      ->withDescription("The filename in Azure Storage. If left empty the filename attribute will be used by default.")
+      ->supportsExpressionLanguage(true)
+      ->build());
+
+
+// DeleteAzureDataLakeStorage
+
+const core::Relationship DeleteAzureDataLakeStorage::Success("success", "If file deletion from Azure storage succeeds the flowfile is transferred to this relationship");
+const core::Relationship DeleteAzureDataLakeStorage::Failure("failure", "If file deletion from Azure storage fails the flowfile is transferred to this relationship");
+
+REGISTER_RESOURCE(DeleteAzureDataLakeStorage, Processor);
+
+
+// FetchAzureDataLakeStorage
+
+const core::Property FetchAzureDataLakeStorage::RangeStart(
+    core::PropertyBuilder::createProperty("Range Start")
+      ->withDescription("The byte position at which to start reading from the object. An empty value or a value of zero will start reading at the beginning of the object.")
+      ->supportsExpressionLanguage(true)
+      ->build());
+
+const core::Property FetchAzureDataLakeStorage::RangeLength(
+    core::PropertyBuilder::createProperty("Range Length")
+      ->withDescription("The number of bytes to download from the object, starting from the Range Start. "
+                        "An empty value or a value that extends beyond the end of the object will read to the end of the object.")
+      ->supportsExpressionLanguage(true)
+      ->build());
+
+const core::Property FetchAzureDataLakeStorage::NumberOfRetries(
+    core::PropertyBuilder::createProperty("Number of Retries")
+      ->withDescription("The number of automatic retries to perform if the download fails.")
+      ->withDefaultValue<uint64_t>(0)
+      ->supportsExpressionLanguage(true)
+      ->build());
+
+const core::Relationship FetchAzureDataLakeStorage::Success("success", "Files that have been successfully fetched from Azure storage are transferred to this relationship");
+const core::Relationship FetchAzureDataLakeStorage::Failure("failure", "In case of fetch failure flowfiles are transferred to this relationship");
+
+REGISTER_RESOURCE(FetchAzureDataLakeStorage, Processor);
+
+
+// PutAzureDataLakeStorage
+
+const core::Property PutAzureDataLakeStorage::ConflictResolutionStrategy(
+    core::PropertyBuilder::createProperty("Conflict Resolution Strategy")
+      ->withDescription("Indicates what should happen when a file with the same name already exists in the output directory.")
+      ->isRequired(true)
+      ->withDefaultValue<std::string>(toString(FileExistsResolutionStrategy::FAIL_FLOW))
+      ->withAllowableValues<std::string>(FileExistsResolutionStrategy::values())
+      ->build());
+
+const core::Relationship PutAzureDataLakeStorage::Success("success", "Files that have been successfully written to Azure storage are transferred to this relationship");
+const core::Relationship PutAzureDataLakeStorage::Failure("failure", "Files that could not be written to Azure storage for some reason are transferred to this relationship");
+
+REGISTER_RESOURCE(PutAzureDataLakeStorage, Processor);
+
+
+// ListAzureDataLakeStorage
+
+const core::Property ListAzureDataLakeStorage::RecurseSubdirectories(
+    core::PropertyBuilder::createProperty("Recurse Subdirectories")
+      ->isRequired(true)
+      ->withDefaultValue<bool>(true)
+      ->withDescription("Indicates whether to list files from subdirectories of the directory")
+      ->build());
+
+const core::Property ListAzureDataLakeStorage::FileFilter(
+  core::PropertyBuilder::createProperty("File Filter")
+    ->withDescription("Only files whose names match the given regular expression will be listed")
+    ->build());
+
+const core::Property ListAzureDataLakeStorage::PathFilter(
+  core::PropertyBuilder::createProperty("Path Filter")
+    ->withDescription("When 'Recurse Subdirectories' is true, then only subdirectories whose paths match the given regular expression will be scanned")
+    ->build());
+
+const core::Property ListAzureDataLakeStorage::ListingStrategy(
+  core::PropertyBuilder::createProperty("Listing Strategy")
+    ->withDescription("Specify how to determine new/updated entities. If 'timestamps' is selected it tracks the latest timestamp of listed entity to "
+                      "determine new/updated entities. If 'none' is selected it lists an entity without any tracking, the same entity will be listed each time on executing this processor.")
+    ->withDefaultValue<std::string>(toString(EntityTracking::TIMESTAMPS))
+    ->withAllowableValues<std::string>(EntityTracking::values())
+    ->build());
+
+const core::Relationship ListAzureDataLakeStorage::Success("success", "All FlowFiles that are received are routed to success");
+
+REGISTER_RESOURCE(ListAzureDataLakeStorage, Processor);
+
+}  // namespace org::apache::nifi::minifi::azure::processors
diff --git a/extensions/azure/processors/DeleteAzureBlobStorage.cpp b/extensions/azure/processors/DeleteAzureBlobStorage.cpp
index 47453b84b..f06026d09 100644
--- a/extensions/azure/processors/DeleteAzureBlobStorage.cpp
+++ b/extensions/azure/processors/DeleteAzureBlobStorage.cpp
@@ -26,37 +26,9 @@
 
 namespace org::apache::nifi::minifi::azure::processors {
 
-const core::Property DeleteAzureBlobStorage::DeleteSnapshotsOption(
-  core::PropertyBuilder::createProperty("Delete Snapshots Option")
-    ->withDescription("Specifies the snapshot deletion options to be used when deleting a blob. None: Deletes the blob only. Include Snapshots: Delete the blob and its snapshots. "
-                      "Delete Snapshots Only: Delete only the blob's snapshots.")
-    ->isRequired(true)
-    ->withDefaultValue<std::string>(toString(storage::OptionalDeletion::NONE))
-    ->withAllowableValues<std::string>(storage::OptionalDeletion::values())
-    ->build());
-
-const core::Relationship DeleteAzureBlobStorage::Success("success", "All successfully processed FlowFiles are routed to this relationship");
-const core::Relationship DeleteAzureBlobStorage::Failure("failure", "Unsuccessful operations will be transferred to the failure relationship");
-
 void DeleteAzureBlobStorage::initialize() {
-  // Set the supported properties
-  setSupportedProperties({
-    AzureStorageCredentialsService,
-    ContainerName,
-    StorageAccountName,
-    StorageAccountKey,
-    SASToken,
-    CommonStorageAccountEndpointSuffix,
-    ConnectionString,
-    Blob,
-    DeleteSnapshotsOption,
-    UseManagedIdentityCredentials
-  });
-  // Set the supported relationships
-  setSupportedRelationships({
-    Success,
-    Failure
-  });
+  setSupportedProperties(properties());
+  setSupportedRelationships(relationships());
 }
 
 void DeleteAzureBlobStorage::onSchedule(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSessionFactory>& session_factory) {
@@ -99,6 +71,4 @@ void DeleteAzureBlobStorage::onTrigger(const std::shared_ptr<core::ProcessContex
   }
 }
 
-REGISTER_RESOURCE(DeleteAzureBlobStorage, "Deletes the provided blob from Azure Storage");
-
 }  // namespace org::apache::nifi::minifi::azure::processors
diff --git a/extensions/azure/processors/DeleteAzureBlobStorage.h b/extensions/azure/processors/DeleteAzureBlobStorage.h
index 526fd11fe..f185094e0 100644
--- a/extensions/azure/processors/DeleteAzureBlobStorage.h
+++ b/extensions/azure/processors/DeleteAzureBlobStorage.h
@@ -29,6 +29,7 @@
 #include "core/Property.h"
 #include "AzureBlobStorageSingleBlobProcessorBase.h"
 #include "core/logging/LoggerConfiguration.h"
+#include "utils/ArrayUtils.h"
 
 template<typename T>
 class AzureBlobStorageTestsFixture;
@@ -37,12 +38,23 @@ namespace org::apache::nifi::minifi::azure::processors {
 
 class DeleteAzureBlobStorage final : public AzureBlobStorageSingleBlobProcessorBase {
  public:
-  // Supported Properties
-  static const core::Property DeleteSnapshotsOption;
+  EXTENSIONAPI static constexpr const char* Description = "Deletes the provided blob from Azure Storage";
 
-  // Supported Relationships
-  static const core::Relationship Failure;
-  static const core::Relationship Success;
+  EXTENSIONAPI static const core::Property DeleteSnapshotsOption;
+  static auto properties() {
+    return utils::array_cat(AzureBlobStorageSingleBlobProcessorBase::properties(), std::array{DeleteSnapshotsOption});
+  }
+
+  EXTENSIONAPI static const core::Relationship Success;
+  EXTENSIONAPI static const core::Relationship Failure;
+  static auto relationships() { return std::array{Success, Failure}; }
+
+  EXTENSIONAPI static constexpr bool SupportsDynamicProperties = false;
+  EXTENSIONAPI static constexpr bool SupportsDynamicRelationships = false;
+  EXTENSIONAPI static constexpr core::annotation::Input InputRequirement = core::annotation::Input::INPUT_REQUIRED;
+  EXTENSIONAPI static constexpr bool IsSingleThreaded = true;
+
+  ADD_COMMON_VIRTUAL_FUNCTIONS_FOR_PROCESSORS
 
   explicit DeleteAzureBlobStorage(const std::string& name, const minifi::utils::Identifier& uuid = minifi::utils::Identifier())
     : DeleteAzureBlobStorage(name, uuid, nullptr) {
@@ -55,14 +67,6 @@ class DeleteAzureBlobStorage final : public AzureBlobStorageSingleBlobProcessorB
  private:
   friend class ::AzureBlobStorageTestsFixture<DeleteAzureBlobStorage>;
 
-  core::annotation::Input getInputRequirement() const override {
-    return core::annotation::Input::INPUT_REQUIRED;
-  }
-
-  bool isSingleThreaded() const override {
-    return true;
-  }
-
   explicit DeleteAzureBlobStorage(const std::string& name, const minifi::utils::Identifier& uuid, std::unique_ptr<storage::BlobStorageClient> blob_storage_client)
     : AzureBlobStorageSingleBlobProcessorBase(name, uuid, core::logging::LoggerFactory<DeleteAzureBlobStorage>::getLogger(), std::move(blob_storage_client)) {
   }
diff --git a/extensions/azure/processors/DeleteAzureDataLakeStorage.cpp b/extensions/azure/processors/DeleteAzureDataLakeStorage.cpp
index eb71d24e2..4bd7fe630 100644
--- a/extensions/azure/processors/DeleteAzureDataLakeStorage.cpp
+++ b/extensions/azure/processors/DeleteAzureDataLakeStorage.cpp
@@ -27,23 +27,9 @@
 
 namespace org::apache::nifi::minifi::azure::processors {
 
-const core::Relationship DeleteAzureDataLakeStorage::Success("success", "If file deletion from Azure storage succeeds the flowfile is transferred to this relationship");
-const core::Relationship DeleteAzureDataLakeStorage::Failure("failure", "If file deletion from Azure storage fails the flowfile is transferred to this relationship");
-
 void DeleteAzureDataLakeStorage::initialize() {
-  // Set the supported properties
-  setSupportedProperties({
-    AzureStorageCredentialsService,
-    FilesystemName,
-    DirectoryName,
-    FileName
-  });
-
-  // Set the supported relationships
-  setSupportedRelationships({
-    Success,
-    Failure
-  });
+  setSupportedProperties(properties());
+  setSupportedRelationships(relationships());
 }
 
 std::optional<storage::DeleteAzureDataLakeStorageParameters> DeleteAzureDataLakeStorage::buildDeleteParameters(
@@ -81,6 +67,4 @@ void DeleteAzureDataLakeStorage::onTrigger(const std::shared_ptr<core::ProcessCo
   }
 }
 
-REGISTER_RESOURCE(DeleteAzureDataLakeStorage, "Deletes the provided file from Azure Data Lake Storage");
-
 }  // namespace org::apache::nifi::minifi::azure::processors
diff --git a/extensions/azure/processors/DeleteAzureDataLakeStorage.h b/extensions/azure/processors/DeleteAzureDataLakeStorage.h
index 3813387fe..89f0e682c 100644
--- a/extensions/azure/processors/DeleteAzureDataLakeStorage.h
+++ b/extensions/azure/processors/DeleteAzureDataLakeStorage.h
@@ -33,9 +33,20 @@ namespace org::apache::nifi::minifi::azure::processors {
 
 class DeleteAzureDataLakeStorage final : public AzureDataLakeStorageFileProcessorBase {
  public:
-  // Supported Relationships
-  static const core::Relationship Failure;
-  static const core::Relationship Success;
+  EXTENSIONAPI static constexpr const char* Description = "Deletes the provided file from Azure Data Lake Storage";
+
+  static auto properties() { return AzureDataLakeStorageFileProcessorBase::properties(); }
+
+  EXTENSIONAPI static const core::Relationship Success;
+  EXTENSIONAPI static const core::Relationship Failure;
+  static auto relationships() { return std::array{Success, Failure}; }
+
+  EXTENSIONAPI static constexpr bool SupportsDynamicProperties = false;
+  EXTENSIONAPI static constexpr bool SupportsDynamicRelationships = false;
+  EXTENSIONAPI static constexpr core::annotation::Input InputRequirement = core::annotation::Input::INPUT_REQUIRED;
+  EXTENSIONAPI static constexpr bool IsSingleThreaded = true;
+
+  ADD_COMMON_VIRTUAL_FUNCTIONS_FOR_PROCESSORS
 
   explicit DeleteAzureDataLakeStorage(const std::string& name, const minifi::utils::Identifier& uuid = minifi::utils::Identifier())
     : AzureDataLakeStorageFileProcessorBase(name, uuid, core::logging::LoggerFactory<DeleteAzureDataLakeStorage>::getLogger()) {
@@ -49,14 +60,6 @@ class DeleteAzureDataLakeStorage final : public AzureDataLakeStorageFileProcesso
  private:
   friend class ::AzureDataLakeStorageTestsFixture<DeleteAzureDataLakeStorage>;
 
-  core::annotation::Input getInputRequirement() const override {
-    return core::annotation::Input::INPUT_REQUIRED;
-  }
-
-  bool isSingleThreaded() const override {
-    return true;
-  }
-
   explicit DeleteAzureDataLakeStorage(const std::string& name, const minifi::utils::Identifier& uuid, std::unique_ptr<storage::DataLakeStorageClient> data_lake_storage_client)
     : AzureDataLakeStorageFileProcessorBase(name, uuid, core::logging::LoggerFactory<DeleteAzureDataLakeStorage>::getLogger(), std::move(data_lake_storage_client)) {
   }
diff --git a/extensions/azure/processors/FetchAzureBlobStorage.cpp b/extensions/azure/processors/FetchAzureBlobStorage.cpp
index c0aded55e..0ed7ab951 100644
--- a/extensions/azure/processors/FetchAzureBlobStorage.cpp
+++ b/extensions/azure/processors/FetchAzureBlobStorage.cpp
@@ -27,39 +27,9 @@
 
 namespace org::apache::nifi::minifi::azure::processors {
 
-const core::Property FetchAzureBlobStorage::RangeStart(
-  core::PropertyBuilder::createProperty("Range Start")
-    ->withDescription("The byte position at which to start reading from the blob. An empty value or a value of zero will start reading at the beginning of the blob.")
-    ->supportsExpressionLanguage(true)
-    ->build());
-const core::Property FetchAzureBlobStorage::RangeLength(
-  core::PropertyBuilder::createProperty("Range Length")
-    ->withDescription("The number of bytes to download from the blob, starting from the Range Start. "
-                      "An empty value or a value that extends beyond the end of the blob will read to the end of the blob.")
-    ->supportsExpressionLanguage(true)
-    ->build());
-
-const core::Relationship FetchAzureBlobStorage::Success("success", "All successfully processed FlowFiles are routed to this relationship");
-const core::Relationship FetchAzureBlobStorage::Failure("failure", "Unsuccessful operations will be transferred to the failure relationship");
-
 void FetchAzureBlobStorage::initialize() {
-  setSupportedProperties({
-    AzureStorageCredentialsService,
-    ContainerName,
-    StorageAccountName,
-    StorageAccountKey,
-    SASToken,
-    CommonStorageAccountEndpointSuffix,
-    ConnectionString,
-    Blob,
-    UseManagedIdentityCredentials,
-    RangeStart,
-    RangeLength
-  });
-  setSupportedRelationships({
-    Success,
-    Failure
-  });
+  setSupportedProperties(properties());
+  setSupportedRelationships(relationships());
 }
 
 std::optional<storage::FetchAzureBlobStorageParameters> FetchAzureBlobStorage::buildFetchAzureBlobStorageParameters(
@@ -119,6 +89,4 @@ void FetchAzureBlobStorage::onTrigger(const std::shared_ptr<core::ProcessContext
   }
 }
 
-REGISTER_RESOURCE(FetchAzureBlobStorage, "Retrieves contents of an Azure Storage Blob, writing the contents to the content of the FlowFile");
-
 }  // namespace org::apache::nifi::minifi::azure::processors
diff --git a/extensions/azure/processors/FetchAzureBlobStorage.h b/extensions/azure/processors/FetchAzureBlobStorage.h
index db295e771..53d238f65 100644
--- a/extensions/azure/processors/FetchAzureBlobStorage.h
+++ b/extensions/azure/processors/FetchAzureBlobStorage.h
@@ -29,6 +29,7 @@
 #include "core/Property.h"
 #include "AzureBlobStorageSingleBlobProcessorBase.h"
 #include "core/logging/LoggerConfiguration.h"
+#include "utils/ArrayUtils.h"
 
 template<typename T>
 class AzureBlobStorageTestsFixture;
@@ -37,11 +38,27 @@ namespace org::apache::nifi::minifi::azure::processors {
 
 class FetchAzureBlobStorage final : public AzureBlobStorageSingleBlobProcessorBase {
  public:
+  EXTENSIONAPI static constexpr const char* Description = "Retrieves contents of an Azure Storage Blob, writing the contents to the content of the FlowFile";
+
   EXTENSIONAPI static const core::Property RangeStart;
   EXTENSIONAPI static const core::Property RangeLength;
+  static auto properties() {
+    return utils::array_cat(AzureBlobStorageSingleBlobProcessorBase::properties(), std::array{
+      RangeStart,
+      RangeLength
+    });
+  }
+
+  EXTENSIONAPI static const core::Relationship Success;
+  EXTENSIONAPI static const core::Relationship Failure;
+  static auto relationships() { return std::array{Success, Failure}; }
 
-  static const core::Relationship Failure;
-  static const core::Relationship Success;
+  EXTENSIONAPI static constexpr bool SupportsDynamicProperties = false;
+  EXTENSIONAPI static constexpr bool SupportsDynamicRelationships = false;
+  EXTENSIONAPI static constexpr core::annotation::Input InputRequirement = core::annotation::Input::INPUT_REQUIRED;
+  EXTENSIONAPI static constexpr bool IsSingleThreaded = false;
+
+  ADD_COMMON_VIRTUAL_FUNCTIONS_FOR_PROCESSORS
 
   explicit FetchAzureBlobStorage(const std::string& name, const minifi::utils::Identifier& uuid = minifi::utils::Identifier())
     : FetchAzureBlobStorage(name, uuid, nullptr) {
@@ -53,10 +70,6 @@ class FetchAzureBlobStorage final : public AzureBlobStorageSingleBlobProcessorBa
  private:
   friend class ::AzureBlobStorageTestsFixture<FetchAzureBlobStorage>;
 
-  core::annotation::Input getInputRequirement() const override {
-    return core::annotation::Input::INPUT_REQUIRED;
-  }
-
   explicit FetchAzureBlobStorage(const std::string& name, const minifi::utils::Identifier& uuid, std::unique_ptr<storage::BlobStorageClient> blob_storage_client)
     : AzureBlobStorageSingleBlobProcessorBase(name, uuid, core::logging::LoggerFactory<FetchAzureBlobStorage>::getLogger(), std::move(blob_storage_client)) {
   }
diff --git a/extensions/azure/processors/FetchAzureDataLakeStorage.cpp b/extensions/azure/processors/FetchAzureDataLakeStorage.cpp
index c27eb19ab..06de9ca8d 100644
--- a/extensions/azure/processors/FetchAzureDataLakeStorage.cpp
+++ b/extensions/azure/processors/FetchAzureDataLakeStorage.cpp
@@ -26,45 +26,9 @@
 
 namespace org::apache::nifi::minifi::azure::processors {
 
-const core::Property FetchAzureDataLakeStorage::RangeStart(
-    core::PropertyBuilder::createProperty("Range Start")
-      ->withDescription("The byte position at which to start reading from the object. An empty value or a value of zero will start reading at the beginning of the object.")
-      ->supportsExpressionLanguage(true)
-      ->build());
-
-const core::Property FetchAzureDataLakeStorage::RangeLength(
-    core::PropertyBuilder::createProperty("Range Length")
-      ->withDescription("The number of bytes to download from the object, starting from the Range Start. "
-                        "An empty value or a value that extends beyond the end of the object will read to the end of the object.")
-      ->supportsExpressionLanguage(true)
-      ->build());
-
-const core::Property FetchAzureDataLakeStorage::NumberOfRetries(
-    core::PropertyBuilder::createProperty("Number of Retries")
-      ->withDescription("The number of automatic retries to perform if the download fails.")
-      ->withDefaultValue<uint64_t>(0)
-      ->supportsExpressionLanguage(true)
-      ->build());
-
-const core::Relationship FetchAzureDataLakeStorage::Success("success", "Files that have been successfully fetched from Azure storage are transferred to this relationship");
-const core::Relationship FetchAzureDataLakeStorage::Failure("failure", "In case of fetch failure flowfiles are transferred to this relationship");
-
 void FetchAzureDataLakeStorage::initialize() {
-  // Add new supported properties
-  setSupportedProperties({
-    AzureStorageCredentialsService,
-    FilesystemName,
-    DirectoryName,
-    FileName,
-    RangeStart,
-    RangeLength,
-    NumberOfRetries
-  });
-  // Set the supported relationships
-  setSupportedRelationships({
-    Success,
-    Failure
-  });
+  setSupportedProperties(properties());
+  setSupportedRelationships(relationships());
 }
 
 std::optional<storage::FetchAzureDataLakeStorageParameters> FetchAzureDataLakeStorage::buildFetchParameters(
@@ -130,6 +94,4 @@ void FetchAzureDataLakeStorage::onTrigger(const std::shared_ptr<core::ProcessCon
   }
 }
 
-REGISTER_RESOURCE(FetchAzureDataLakeStorage, "Fetch the provided file from Azure Data Lake Storage Gen 2");
-
 }  // namespace org::apache::nifi::minifi::azure::processors
diff --git a/extensions/azure/processors/FetchAzureDataLakeStorage.h b/extensions/azure/processors/FetchAzureDataLakeStorage.h
index ec84b699b..7e9cd295a 100644
--- a/extensions/azure/processors/FetchAzureDataLakeStorage.h
+++ b/extensions/azure/processors/FetchAzureDataLakeStorage.h
@@ -1,7 +1,4 @@
 /**
- * @file FetchAzureDataLakeStorage.h
- * FetchAzureDataLakeStorage class declaration
- *
  * 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.
@@ -25,6 +22,7 @@
 #include <memory>
 
 #include "io/StreamPipe.h"
+#include "utils/ArrayUtils.h"
 #include "AzureDataLakeStorageFileProcessorBase.h"
 
 template<typename AzureDataLakeStorageProcessor>
@@ -34,14 +32,29 @@ namespace org::apache::nifi::minifi::azure::processors {
 
 class FetchAzureDataLakeStorage final : public AzureDataLakeStorageFileProcessorBase {
  public:
-  // Supported Properties
+  EXTENSIONAPI static constexpr const char* Description = "Fetch the provided file from Azure Data Lake Storage Gen 2";
+
   EXTENSIONAPI static const core::Property RangeStart;
   EXTENSIONAPI static const core::Property RangeLength;
   EXTENSIONAPI static const core::Property NumberOfRetries;
+  static auto properties() {
+    return utils::array_cat(AzureDataLakeStorageFileProcessorBase::properties(), std::array{
+      RangeStart,
+      RangeLength,
+      NumberOfRetries
+    });
+  }
+
+  EXTENSIONAPI static const core::Relationship Success;
+  EXTENSIONAPI static const core::Relationship Failure;
+  static auto relationships() { return std::array{Success, Failure}; }
+
+  EXTENSIONAPI static constexpr bool SupportsDynamicProperties = false;
+  EXTENSIONAPI static constexpr bool SupportsDynamicRelationships = false;
+  EXTENSIONAPI static constexpr core::annotation::Input InputRequirement = core::annotation::Input::INPUT_REQUIRED;
+  EXTENSIONAPI static constexpr bool IsSingleThreaded = true;
 
-  // Supported Relationships
-  static const core::Relationship Failure;
-  static const core::Relationship Success;
+  ADD_COMMON_VIRTUAL_FUNCTIONS_FOR_PROCESSORS
 
   explicit FetchAzureDataLakeStorage(const std::string& name, const minifi::utils::Identifier& uuid = minifi::utils::Identifier())
     : AzureDataLakeStorageFileProcessorBase(name, uuid, core::logging::LoggerFactory<FetchAzureDataLakeStorage>::getLogger()) {
@@ -55,14 +68,6 @@ class FetchAzureDataLakeStorage final : public AzureDataLakeStorageFileProcessor
  private:
   friend class ::AzureDataLakeStorageTestsFixture<FetchAzureDataLakeStorage>;
 
-  core::annotation::Input getInputRequirement() const override {
-    return core::annotation::Input::INPUT_REQUIRED;
-  }
-
-  bool isSingleThreaded() const override {
-    return true;
-  }
-
   explicit FetchAzureDataLakeStorage(const std::string& name, const minifi::utils::Identifier& uuid, std::unique_ptr<storage::DataLakeStorageClient> data_lake_storage_client)
     : AzureDataLakeStorageFileProcessorBase(name, uuid, core::logging::LoggerFactory<FetchAzureDataLakeStorage>::getLogger(), std::move(data_lake_storage_client)) {
   }
diff --git a/extensions/azure/processors/ListAzureBlobStorage.cpp b/extensions/azure/processors/ListAzureBlobStorage.cpp
index dd9940525..c08f71ab2 100644
--- a/extensions/azure/processors/ListAzureBlobStorage.cpp
+++ b/extensions/azure/processors/ListAzureBlobStorage.cpp
@@ -26,41 +26,9 @@
 
 namespace org::apache::nifi::minifi::azure::processors {
 
-const core::Property ListAzureBlobStorage::ListingStrategy(
-  core::PropertyBuilder::createProperty("Listing Strategy")
-    ->withDescription("Specify how to determine new/updated entities. If 'timestamps' is selected it tracks the latest timestamp of listed entity to determine new/updated entities. "
-                      "If 'none' is selected it lists an entity without any tracking, the same entity will be listed each time on executing this processor.")
-    ->isRequired(true)
-    ->withDefaultValue<std::string>(toString(EntityTracking::TIMESTAMPS))
-    ->withAllowableValues<std::string>(EntityTracking::values())
-    ->build());
-
-const core::Property ListAzureBlobStorage::Prefix(
-  core::PropertyBuilder::createProperty("Prefix")
-    ->withDescription("Search prefix for listing")
-    ->supportsExpressionLanguage(true)
-    ->build());
-
-const core::Relationship ListAzureBlobStorage::Success("success", "All FlowFiles that are received are routed to success");
-
 void ListAzureBlobStorage::initialize() {
-  // Set the supported properties
-  setSupportedProperties({
-    AzureStorageCredentialsService,
-    ContainerName,
-    StorageAccountName,
-    StorageAccountKey,
-    SASToken,
-    CommonStorageAccountEndpointSuffix,
-    ConnectionString,
-    UseManagedIdentityCredentials,
-    ListingStrategy,
-    Prefix
-  });
-  // Set the supported relationships
-  setSupportedRelationships({
-    Success
-  });
+  setSupportedProperties(properties());
+  setSupportedRelationships(relationships());
 }
 
 void ListAzureBlobStorage::onSchedule(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSessionFactory>& session_factory) {
@@ -145,6 +113,4 @@ void ListAzureBlobStorage::onTrigger(const std::shared_ptr<core::ProcessContext>
   }
 }
 
-REGISTER_RESOURCE(ListAzureBlobStorage, "Lists blobs in an Azure Storage container. Listing details are attached to an empty FlowFile for use with FetchAzureBlobStorage.");
-
 }  // namespace org::apache::nifi::minifi::azure::processors
diff --git a/extensions/azure/processors/ListAzureBlobStorage.h b/extensions/azure/processors/ListAzureBlobStorage.h
index ab1b07e3e..18740fabc 100644
--- a/extensions/azure/processors/ListAzureBlobStorage.h
+++ b/extensions/azure/processors/ListAzureBlobStorage.h
@@ -39,12 +39,26 @@ class ListAzureBlobStorage final : public AzureBlobStorageProcessorBase {
     (TIMESTAMPS, "timestamps")
   )
 
-  // Supported Properties
+  EXTENSIONAPI static constexpr const char* Description = "Lists blobs in an Azure Storage container. Listing details are attached to an empty FlowFile for use with FetchAzureBlobStorage.";
+
   EXTENSIONAPI static const core::Property ListingStrategy;
   EXTENSIONAPI static const core::Property Prefix;
+  static auto properties() {
+    return utils::array_cat(AzureBlobStorageProcessorBase::properties(), std::array{
+      ListingStrategy,
+      Prefix
+    });
+  }
+
+  EXTENSIONAPI static const core::Relationship Success;
+  static auto relationships() { return std::array{Success}; }
 
-  // Supported Relationships
-  static const core::Relationship Success;
+  EXTENSIONAPI static constexpr bool SupportsDynamicProperties = false;
+  EXTENSIONAPI static constexpr bool SupportsDynamicRelationships = false;
+  EXTENSIONAPI static constexpr core::annotation::Input InputRequirement = core::annotation::Input::INPUT_FORBIDDEN;
+  EXTENSIONAPI static constexpr bool IsSingleThreaded = false;
+
+  ADD_COMMON_VIRTUAL_FUNCTIONS_FOR_PROCESSORS
 
   explicit ListAzureBlobStorage(const std::string& name, const minifi::utils::Identifier& uuid = minifi::utils::Identifier())
     : ListAzureBlobStorage(name, nullptr, uuid) {
@@ -59,10 +73,6 @@ class ListAzureBlobStorage final : public AzureBlobStorageProcessorBase {
   void onTrigger(const std::shared_ptr<core::ProcessContext> &context, const std::shared_ptr<core::ProcessSession> &session) override;
 
  private:
-  core::annotation::Input getInputRequirement() const override {
-    return core::annotation::Input::INPUT_FORBIDDEN;
-  }
-
   std::optional<storage::ListAzureBlobStorageParameters> buildListAzureBlobStorageParameters(core::ProcessContext &context);
   std::shared_ptr<core::FlowFile> createNewFlowFile(core::ProcessSession &session, const storage::ListContainerResultElement &element);
 
diff --git a/extensions/azure/processors/ListAzureDataLakeStorage.cpp b/extensions/azure/processors/ListAzureDataLakeStorage.cpp
index e284e9ca0..4dfc4e1bb 100644
--- a/extensions/azure/processors/ListAzureDataLakeStorage.cpp
+++ b/extensions/azure/processors/ListAzureDataLakeStorage.cpp
@@ -26,33 +26,6 @@
 
 namespace org::apache::nifi::minifi::azure::processors {
 
-const core::Property ListAzureDataLakeStorage::RecurseSubdirectories(
-    core::PropertyBuilder::createProperty("Recurse Subdirectories")
-      ->isRequired(true)
-      ->withDefaultValue<bool>(true)
-      ->withDescription("Indicates whether to list files from subdirectories of the directory")
-      ->build());
-
-const core::Property ListAzureDataLakeStorage::FileFilter(
-  core::PropertyBuilder::createProperty("File Filter")
-    ->withDescription("Only files whose names match the given regular expression will be listed")
-    ->build());
-
-const core::Property ListAzureDataLakeStorage::PathFilter(
-  core::PropertyBuilder::createProperty("Path Filter")
-    ->withDescription("When 'Recurse Subdirectories' is true, then only subdirectories whose paths match the given regular expression will be scanned")
-    ->build());
-
-const core::Property ListAzureDataLakeStorage::ListingStrategy(
-  core::PropertyBuilder::createProperty("Listing Strategy")
-    ->withDescription("Specify how to determine new/updated entities. If 'timestamps' is selected it tracks the latest timestamp of listed entity to "
-                      "determine new/updated entities. If 'none' is selected it lists an entity without any tracking, the same entity will be listed each time on executing this processor.")
-    ->withDefaultValue<std::string>(toString(EntityTracking::TIMESTAMPS))
-    ->withAllowableValues<std::string>(EntityTracking::values())
-    ->build());
-
-const core::Relationship ListAzureDataLakeStorage::Success("success", "All FlowFiles that are received are routed to success");
-
 namespace {
 std::shared_ptr<core::FlowFile> createNewFlowFile(core::ProcessSession &session, const storage::ListDataLakeStorageElement &element) {
   auto flow_file = session.create();
@@ -68,18 +41,8 @@ std::shared_ptr<core::FlowFile> createNewFlowFile(core::ProcessSession &session,
 }  // namespace
 
 void ListAzureDataLakeStorage::initialize() {
-  setSupportedProperties({
-    AzureStorageCredentialsService,
-    FilesystemName,
-    DirectoryName,
-    RecurseSubdirectories,
-    FileFilter,
-    PathFilter,
-    ListingStrategy
-  });
-  setSupportedRelationships({
-    Success
-  });
+  setSupportedProperties(properties());
+  setSupportedRelationships(relationships());
 }
 
 void ListAzureDataLakeStorage::onSchedule(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSessionFactory>& sessionFactory) {
@@ -168,6 +131,4 @@ void ListAzureDataLakeStorage::onTrigger(const std::shared_ptr<core::ProcessCont
   }
 }
 
-REGISTER_RESOURCE(ListAzureDataLakeStorage, "Lists directory in an Azure Data Lake Storage Gen 2 filesystem");
-
 }  // namespace org::apache::nifi::minifi::azure::processors
diff --git a/extensions/azure/processors/ListAzureDataLakeStorage.h b/extensions/azure/processors/ListAzureDataLakeStorage.h
index 51bd17c78..8ae04ba3d 100644
--- a/extensions/azure/processors/ListAzureDataLakeStorage.h
+++ b/extensions/azure/processors/ListAzureDataLakeStorage.h
@@ -25,6 +25,7 @@
 #include <memory>
 
 #include "AzureDataLakeStorageProcessorBase.h"
+#include "utils/ArrayUtils.h"
 
 class ListAzureDataLakeStorageTestsFixture;
 
@@ -37,12 +38,30 @@ class ListAzureDataLakeStorage final : public AzureDataLakeStorageProcessorBase
     (TIMESTAMPS, "timestamps")
   )
 
+  EXTENSIONAPI static constexpr const char* Description = "Lists directory in an Azure Data Lake Storage Gen 2 filesystem";
+
   EXTENSIONAPI static const core::Property RecurseSubdirectories;
   EXTENSIONAPI static const core::Property FileFilter;
   EXTENSIONAPI static const core::Property PathFilter;
   EXTENSIONAPI static const core::Property ListingStrategy;
+  static auto properties() {
+    return utils::array_cat(AzureDataLakeStorageProcessorBase::properties(), std::array{
+      RecurseSubdirectories,
+      FileFilter,
+      PathFilter,
+      ListingStrategy
+    });
+  }
+
+  EXTENSIONAPI static const core::Relationship Success;
+  static auto relationships() { return std::array{Success}; }
 
-  static const core::Relationship Success;
+  EXTENSIONAPI static constexpr bool SupportsDynamicProperties = false;
+  EXTENSIONAPI static constexpr bool SupportsDynamicRelationships = false;
+  EXTENSIONAPI static constexpr core::annotation::Input InputRequirement = core::annotation::Input::INPUT_FORBIDDEN;
+  EXTENSIONAPI static constexpr bool IsSingleThreaded = false;
+
+  ADD_COMMON_VIRTUAL_FUNCTIONS_FOR_PROCESSORS
 
   explicit ListAzureDataLakeStorage(const std::string& name, const minifi::utils::Identifier& uuid = minifi::utils::Identifier())
     : AzureDataLakeStorageProcessorBase(name, uuid, core::logging::LoggerFactory<ListAzureDataLakeStorage>::getLogger()) {
@@ -57,10 +76,6 @@ class ListAzureDataLakeStorage final : public AzureDataLakeStorageProcessorBase
  private:
   friend class ::ListAzureDataLakeStorageTestsFixture;
 
-  core::annotation::Input getInputRequirement() const override {
-    return core::annotation::Input::INPUT_FORBIDDEN;
-  }
-
   explicit ListAzureDataLakeStorage(const std::string& name, const minifi::utils::Identifier& uuid, std::unique_ptr<storage::DataLakeStorageClient> data_lake_storage_client)
     : AzureDataLakeStorageProcessorBase(name, uuid, core::logging::LoggerFactory<ListAzureDataLakeStorage>::getLogger(), std::move(data_lake_storage_client)) {
   }
diff --git a/extensions/azure/processors/PutAzureBlobStorage.cpp b/extensions/azure/processors/PutAzureBlobStorage.cpp
index 493fa93e3..d07225344 100644
--- a/extensions/azure/processors/PutAzureBlobStorage.cpp
+++ b/extensions/azure/processors/PutAzureBlobStorage.cpp
@@ -26,37 +26,9 @@
 
 namespace org::apache::nifi::minifi::azure::processors {
 
-const core::Property PutAzureBlobStorage::CreateContainer(
-  core::PropertyBuilder::createProperty("Create Container")
-    ->withDescription("Specifies whether to check if the container exists and to automatically create it if it does not. "
-                      "Permission to list containers is required. If false, this check is not made, but the Put operation will "
-                      "fail if the container does not exist.")
-    ->isRequired(true)
-    ->withDefaultValue<bool>(false)
-    ->build());
-
-const core::Relationship PutAzureBlobStorage::Success("success", "All successfully processed FlowFiles are routed to this relationship");
-const core::Relationship PutAzureBlobStorage::Failure("failure", "Unsuccessful operations will be transferred to the failure relationship");
-
 void PutAzureBlobStorage::initialize() {
-  // Set the supported properties
-  setSupportedProperties({
-    AzureStorageCredentialsService,
-    ContainerName,
-    StorageAccountName,
-    StorageAccountKey,
-    SASToken,
-    CommonStorageAccountEndpointSuffix,
-    ConnectionString,
-    Blob,
-    CreateContainer,
-    UseManagedIdentityCredentials
-  });
-  // Set the supported relationships
-  setSupportedRelationships({
-    Success,
-    Failure
-  });
+  setSupportedProperties(properties());
+  setSupportedRelationships(relationships());
 }
 
 
@@ -118,6 +90,4 @@ void PutAzureBlobStorage::onTrigger(const std::shared_ptr<core::ProcessContext>
   session->transfer(flow_file, Success);
 }
 
-REGISTER_RESOURCE(PutAzureBlobStorage, "Puts content into an Azure Storage Blob");
-
 }  // namespace org::apache::nifi::minifi::azure::processors
diff --git a/extensions/azure/processors/PutAzureBlobStorage.h b/extensions/azure/processors/PutAzureBlobStorage.h
index ccb731af3..619269f04 100644
--- a/extensions/azure/processors/PutAzureBlobStorage.h
+++ b/extensions/azure/processors/PutAzureBlobStorage.h
@@ -30,6 +30,7 @@
 #include "core/logging/LoggerConfiguration.h"
 #include "AzureBlobStorageSingleBlobProcessorBase.h"
 #include "io/StreamPipe.h"
+#include "utils/ArrayUtils.h"
 
 template<typename T>
 class AzureBlobStorageTestsFixture;
@@ -38,12 +39,23 @@ namespace org::apache::nifi::minifi::azure::processors {
 
 class PutAzureBlobStorage final : public AzureBlobStorageSingleBlobProcessorBase {
  public:
-  // Supported Properties
-  static const core::Property CreateContainer;
+  EXTENSIONAPI static constexpr const char* Description = "Puts content into an Azure Storage Blob";
 
-  // Supported Relationships
-  static const core::Relationship Failure;
-  static const core::Relationship Success;
+  EXTENSIONAPI static const core::Property CreateContainer;
+  static auto properties() {
+    return utils::array_cat(AzureBlobStorageSingleBlobProcessorBase::properties(), std::array{CreateContainer});
+  }
+
+  EXTENSIONAPI static const core::Relationship Success;
+  EXTENSIONAPI static const core::Relationship Failure;
+  static auto relationships() { return std::array{Success, Failure}; }
+
+  EXTENSIONAPI static constexpr bool SupportsDynamicProperties = false;
+  EXTENSIONAPI static constexpr bool SupportsDynamicRelationships = false;
+  EXTENSIONAPI static constexpr core::annotation::Input InputRequirement = core::annotation::Input::INPUT_ALLOWED;
+  EXTENSIONAPI static constexpr bool IsSingleThreaded = false;
+
+  ADD_COMMON_VIRTUAL_FUNCTIONS_FOR_PROCESSORS
 
   explicit PutAzureBlobStorage(const std::string& name, const minifi::utils::Identifier& uuid = minifi::utils::Identifier())
     : PutAzureBlobStorage(name, uuid, nullptr) {
diff --git a/extensions/azure/processors/PutAzureDataLakeStorage.cpp b/extensions/azure/processors/PutAzureDataLakeStorage.cpp
index 7df24ec4a..ccf5c6754 100644
--- a/extensions/azure/processors/PutAzureDataLakeStorage.cpp
+++ b/extensions/azure/processors/PutAzureDataLakeStorage.cpp
@@ -29,31 +29,9 @@
 
 namespace org::apache::nifi::minifi::azure::processors {
 
-const core::Property PutAzureDataLakeStorage::ConflictResolutionStrategy(
-    core::PropertyBuilder::createProperty("Conflict Resolution Strategy")
-      ->withDescription("Indicates what should happen when a file with the same name already exists in the output directory.")
-      ->isRequired(true)
-      ->withDefaultValue<std::string>(toString(FileExistsResolutionStrategy::FAIL_FLOW))
-      ->withAllowableValues<std::string>(FileExistsResolutionStrategy::values())
-      ->build());
-
-const core::Relationship PutAzureDataLakeStorage::Success("success", "Files that have been successfully written to Azure storage are transferred to this relationship");
-const core::Relationship PutAzureDataLakeStorage::Failure("failure", "Files that could not be written to Azure storage for some reason are transferred to this relationship");
-
 void PutAzureDataLakeStorage::initialize() {
-  // Set the supported properties
-  setSupportedProperties({
-    AzureStorageCredentialsService,
-    FilesystemName,
-    DirectoryName,
-    FileName,
-    ConflictResolutionStrategy
-  });
-  // Set the supported relationships
-  setSupportedRelationships({
-    Success,
-    Failure
-  });
+  setSupportedProperties(properties());
+  setSupportedRelationships(relationships());
 }
 
 void PutAzureDataLakeStorage::onSchedule(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSessionFactory>& sessionFactory) {
@@ -150,6 +128,4 @@ int64_t PutAzureDataLakeStorage::ReadCallback::operator()(const std::shared_ptr<
   return read_ret;
 }
 
-REGISTER_RESOURCE(PutAzureDataLakeStorage, "Puts content into an Azure Data Lake Storage Gen 2");
-
 }  // namespace org::apache::nifi::minifi::azure::processors
diff --git a/extensions/azure/processors/PutAzureDataLakeStorage.h b/extensions/azure/processors/PutAzureDataLakeStorage.h
index d6d312892..420175a27 100644
--- a/extensions/azure/processors/PutAzureDataLakeStorage.h
+++ b/extensions/azure/processors/PutAzureDataLakeStorage.h
@@ -26,7 +26,7 @@
 
 #include "AzureDataLakeStorageFileProcessorBase.h"
 #include "io/StreamPipe.h"
-
+#include "utils/ArrayUtils.h"
 #include "utils/Enum.h"
 #include "utils/Export.h"
 
@@ -37,12 +37,23 @@ namespace org::apache::nifi::minifi::azure::processors {
 
 class PutAzureDataLakeStorage final : public AzureDataLakeStorageFileProcessorBase {
  public:
-  // Supported Properties
+  EXTENSIONAPI static constexpr const char* Description = "Puts content into an Azure Data Lake Storage Gen 2";
+
   EXTENSIONAPI static const core::Property ConflictResolutionStrategy;
+  static auto properties() {
+    return utils::array_cat(AzureDataLakeStorageFileProcessorBase::properties(), std::array{ConflictResolutionStrategy});
+  }
 
-  // Supported Relationships
-  EXTENSIONAPI static const core::Relationship Failure;
   EXTENSIONAPI static const core::Relationship Success;
+  EXTENSIONAPI static const core::Relationship Failure;
+  static auto relationships() { return std::array{Success, Failure}; }
+
+  EXTENSIONAPI static constexpr bool SupportsDynamicProperties = false;
+  EXTENSIONAPI static constexpr bool SupportsDynamicRelationships = false;
+  EXTENSIONAPI static constexpr core::annotation::Input InputRequirement = core::annotation::Input::INPUT_REQUIRED;
+  EXTENSIONAPI static constexpr bool IsSingleThreaded = true;
+
+  ADD_COMMON_VIRTUAL_FUNCTIONS_FOR_PROCESSORS
 
   SMART_ENUM(FileExistsResolutionStrategy,
     (FAIL_FLOW, "fail"),
@@ -78,14 +89,6 @@ class PutAzureDataLakeStorage final : public AzureDataLakeStorageFileProcessorBa
     std::shared_ptr<core::logging::Logger> logger_;
   };
 
-  core::annotation::Input getInputRequirement() const override {
-    return core::annotation::Input::INPUT_REQUIRED;
-  }
-
-  bool isSingleThreaded() const override {
-    return true;
-  }
-
   explicit PutAzureDataLakeStorage(const std::string& name, const minifi::utils::Identifier& uuid, std::unique_ptr<storage::DataLakeStorageClient> data_lake_storage_client)
     : AzureDataLakeStorageFileProcessorBase(name, uuid, core::logging::LoggerFactory<PutAzureDataLakeStorage>::getLogger(), std::move(data_lake_storage_client)) {
   }
diff --git a/extensions/bustache/ApplyTemplate.cpp b/extensions/bustache/ApplyTemplate.cpp
index 4b2041566..57274b391 100644
--- a/extensions/bustache/ApplyTemplate.cpp
+++ b/extensions/bustache/ApplyTemplate.cpp
@@ -35,8 +35,8 @@ const core::Property ApplyTemplate::Template("Template", "Path to the input must
 const core::Relationship ApplyTemplate::Success("success", "success operational on the flow record");
 
 void ApplyTemplate::initialize() {
-  setSupportedProperties({Template});
-  setSupportedRelationships({Success});
+  setSupportedProperties(properties());
+  setSupportedRelationships(relationships());
 }
 
 void ApplyTemplate::onTrigger(const std::shared_ptr<core::ProcessContext> &context,
@@ -72,7 +72,6 @@ void ApplyTemplate::onTrigger(const std::shared_ptr<core::ProcessContext> &conte
   session->transfer(flow_file, Success);
 }
 
-REGISTER_RESOURCE(ApplyTemplate, "Applies the mustache template specified by the \"Template\" property and writes the output to the flow file content. "
-    "FlowFile attributes are used as template parameters.");
+REGISTER_RESOURCE(ApplyTemplate, Processor);
 
 }  // namespace org::apache::nifi::minifi::processors
diff --git a/extensions/bustache/ApplyTemplate.h b/extensions/bustache/ApplyTemplate.h
index e0033172b..bcafb82cd 100644
--- a/extensions/bustache/ApplyTemplate.h
+++ b/extensions/bustache/ApplyTemplate.h
@@ -35,11 +35,22 @@ class ApplyTemplate : public core::Processor {
  public:
   explicit ApplyTemplate(const std::string& name, const utils::Identifier& uuid = {})
       : Processor(name, uuid) {}
-  static constexpr char const *ProcessorName = "ApplyTemplate";
 
-  static const core::Property Template;
+  EXTENSIONAPI static constexpr const char* Description = "Applies the mustache template specified by the \"Template\" property and writes the output to the flow file content. "
+    "FlowFile attributes are used as template parameters.";
 
-  static const core::Relationship Success;
+  EXTENSIONAPI static const core::Property Template;
+  static auto properties() { return std::array{Template}; }
+
+  EXTENSIONAPI static const core::Relationship Success;
+  static auto relationships() { return std::array{Success}; }
+
+  EXTENSIONAPI static constexpr bool SupportsDynamicProperties = false;
+  EXTENSIONAPI static constexpr bool SupportsDynamicRelationships = false;
+  EXTENSIONAPI static constexpr core::annotation::Input InputRequirement = core::annotation::Input::INPUT_ALLOWED;
+  EXTENSIONAPI static constexpr bool IsSingleThreaded = false;
+
+  ADD_COMMON_VIRTUAL_FUNCTIONS_FOR_PROCESSORS
 
   void onTrigger(const std::shared_ptr<core::ProcessContext> &context, const std::shared_ptr<core::ProcessSession> &session) override;
   void initialize() override;
diff --git a/extensions/civetweb/processors/ListenHTTP.cpp b/extensions/civetweb/processors/ListenHTTP.cpp
index 52a0ae55e..a7f99b675 100644
--- a/extensions/civetweb/processors/ListenHTTP.cpp
+++ b/extensions/civetweb/processors/ListenHTTP.cpp
@@ -26,6 +26,7 @@
 #include <utility>
 #include <vector>
 
+#include "core/PropertyBuilder.h"
 #include "core/Resource.h"
 #include "utils/gsl.h"
 
@@ -37,75 +38,60 @@ namespace processors {
 
 const uint64_t ListenHTTP::DEFAULT_BUFFER_SIZE = 20000;
 
-core::Property ListenHTTP::BasePath(
+const core::Property ListenHTTP::BasePath(
     core::PropertyBuilder::createProperty("Base Path")
         ->withDescription("Base path for incoming connections")
         ->isRequired(false)
         ->withDefaultValue<std::string>("contentListener")->build());
 
-core::Property ListenHTTP::Port(
+const core::Property ListenHTTP::Port(
     core::PropertyBuilder::createProperty("Listening Port")
         ->withDescription("The Port to listen on for incoming connections. 0 means port is going to be selected randomly.")
         ->isRequired(true)
         ->withDefaultValue<int>(80, core::StandardValidators::get().LISTEN_PORT_VALIDATOR)->build());
 
-core::Property ListenHTTP::AuthorizedDNPattern("Authorized DN Pattern", "A Regular Expression to apply against the Distinguished Name of incoming"
+const core::Property ListenHTTP::AuthorizedDNPattern("Authorized DN Pattern", "A Regular Expression to apply against the Distinguished Name of incoming"
                                                " connections. If the Pattern does not match the DN, the connection will be refused.",
                                                ".*");
-core::Property ListenHTTP::SSLCertificate("SSL Certificate", "File containing PEM-formatted file including TLS/SSL certificate and key", "");
-core::Property ListenHTTP::SSLCertificateAuthority("SSL Certificate Authority", "File containing trusted PEM-formatted certificates", "");
+const core::Property ListenHTTP::SSLCertificate("SSL Certificate", "File containing PEM-formatted file including TLS/SSL certificate and key", "");
+const core::Property ListenHTTP::SSLCertificateAuthority("SSL Certificate Authority", "File containing trusted PEM-formatted certificates", "");
 
-core::Property ListenHTTP::SSLVerifyPeer(
+const core::Property ListenHTTP::SSLVerifyPeer(
     core::PropertyBuilder::createProperty("SSL Verify Peer")
         ->withDescription("Whether or not to verify the client's certificate (yes/no)")
         ->isRequired(false)
         ->withAllowableValues<std::string>({"yes", "no"})
         ->withDefaultValue("no")->build());
 
-core::Property ListenHTTP::SSLMinimumVersion(
+const core::Property ListenHTTP::SSLMinimumVersion(
     core::PropertyBuilder::createProperty("SSL Minimum Version")
         ->withDescription("Minimum TLS/SSL version allowed (TLS1.2)")
         ->isRequired(false)
         ->withAllowableValues<std::string>({"TLS1.2"})
         ->withDefaultValue("TLS1.2")->build());
 
-core::Property ListenHTTP::HeadersAsAttributesRegex("HTTP Headers to receive as Attributes (Regex)", "Specifies the Regular Expression that determines the names of HTTP Headers that"
+const core::Property ListenHTTP::HeadersAsAttributesRegex("HTTP Headers to receive as Attributes (Regex)", "Specifies the Regular Expression that determines the names of HTTP Headers that"
                                                     " should be passed along as FlowFile attributes",
                                                     "");
 
-core::Property ListenHTTP::BatchSize(
+const core::Property ListenHTTP::BatchSize(
     core::PropertyBuilder::createProperty("Batch Size")
         ->withDescription("Maximum number of buffered requests to be processed in a single batch. If set to zero all buffered requests are processed.")
         ->withDefaultValue<uint64_t>(ListenHTTP::DEFAULT_BUFFER_SIZE)->build());
 
-core::Property ListenHTTP::BufferSize(
+const core::Property ListenHTTP::BufferSize(
     core::PropertyBuilder::createProperty("Buffer Size")
         ->withDescription("Maximum number of HTTP Requests allowed to be buffered before processing them when the processor is triggered. "
                           "If the buffer full, the request is refused. If set to zero the buffer is unlimited.")
         ->withDefaultValue<uint64_t>(ListenHTTP::DEFAULT_BUFFER_SIZE)->build());
 
-core::Relationship ListenHTTP::Success("success", "All files are routed to success");
+const core::Relationship ListenHTTP::Success("success", "All files are routed to success");
 
 void ListenHTTP::initialize() {
   logger_->log_trace("Initializing ListenHTTP");
 
-  // Set the supported properties
-  std::set<core::Property> properties;
-  properties.insert(BasePath);
-  properties.insert(Port);
-  properties.insert(AuthorizedDNPattern);
-  properties.insert(SSLCertificate);
-  properties.insert(SSLCertificateAuthority);
-  properties.insert(SSLVerifyPeer);
-  properties.insert(SSLMinimumVersion);
-  properties.insert(HeadersAsAttributesRegex);
-  properties.insert(BatchSize);
-  properties.insert(BufferSize);
-  setSupportedProperties(properties);
-  // Set the supported relationships
-  std::set<core::Relationship> relationships;
-  relationships.insert(Success);
-  setSupportedRelationships(relationships);
+  setSupportedProperties(properties());
+  setSupportedRelationships(relationships());
 }
 
 void ListenHTTP::onSchedule(core::ProcessContext *context, core::ProcessSessionFactory* /*sessionFactory*/) {
@@ -522,12 +508,7 @@ void ListenHTTP::notifyStop() {
   handler_.reset();
 }
 
-REGISTER_RESOURCE(ListenHTTP, "Starts an HTTP Server and listens on a given base path to transform incoming requests into FlowFiles. The default URI of the Service will be "
-    "http://{hostname}:{port}/contentListener. Only HEAD, POST, and GET requests are supported. PUT, and DELETE will result in an error and the HTTP response status code 405."
-    " The response body text for all requests, by default, is empty (length of 0). A static response body can be set for a given URI by sending input files to ListenHTTP with "
-    "the http.type attribute set to response_body. The response body FlowFile filename attribute is appended to the Base Path property (separated by a /) when mapped to incoming requests. "
-    "The mime.type attribute of the response body FlowFile is used for the Content-type header in responses. Response body content can be cleared by sending an empty (size 0) "
-    "FlowFile for a given URI mapping.");
+REGISTER_RESOURCE(ListenHTTP, Processor);
 
 } /* namespace processors */
 } /* namespace minifi */
diff --git a/extensions/civetweb/processors/ListenHTTP.h b/extensions/civetweb/processors/ListenHTTP.h
index 683fc0dc8..cefd2da1b 100644
--- a/extensions/civetweb/processors/ListenHTTP.h
+++ b/extensions/civetweb/processors/ListenHTTP.h
@@ -42,38 +42,59 @@ namespace nifi {
 namespace minifi {
 namespace processors {
 
-// ListenHTTP Class
 class ListenHTTP : public core::Processor {
  public:
   using FlowFileBufferPair = std::pair<std::shared_ptr<FlowFileRecord>, std::unique_ptr<io::BufferStream>>;
 
-  // Constructor
-  /*!
-   * Create a new processor
-   */
   explicit ListenHTTP(const std::string& name, const utils::Identifier& uuid = {})
       : Processor(name, uuid),
         batch_size_(0) {
     callbacks_.log_message = &logMessage;
     callbacks_.log_access = &logAccess;
   }
-  // Destructor
   ~ListenHTTP() override;
-  // Processor Name
-  EXTENSIONAPI static constexpr char const *ProcessorName = "ListenHTTP";
-  // Supported Properties
-  EXTENSIONAPI static core::Property BasePath;
-  EXTENSIONAPI static core::Property Port;
-  EXTENSIONAPI static core::Property AuthorizedDNPattern;
-  EXTENSIONAPI static core::Property SSLCertificate;
-  EXTENSIONAPI static core::Property SSLCertificateAuthority;
-  EXTENSIONAPI static core::Property SSLVerifyPeer;
-  EXTENSIONAPI static core::Property SSLMinimumVersion;
-  EXTENSIONAPI static core::Property HeadersAsAttributesRegex;
-  EXTENSIONAPI static core::Property BatchSize;
-  EXTENSIONAPI static core::Property BufferSize;
-  // Supported Relationships
-  EXTENSIONAPI static core::Relationship Success;
+
+  EXTENSIONAPI static constexpr const char* Description = "Starts an HTTP Server and listens on a given base path to transform incoming requests into FlowFiles. The default URI of the Service "
+      "will be http://{hostname}:{port}/contentListener. Only HEAD, POST, and GET requests are supported. PUT, and DELETE will result in an error and the HTTP response status code 405. "
+      "The response body text for all requests, by default, is empty (length of 0). A static response body can be set for a given URI by sending input files to ListenHTTP with "
+      "the http.type attribute set to response_body. The response body FlowFile filename attribute is appended to the Base Path property (separated by a /) when mapped to incoming requests. "
+      "The mime.type attribute of the response body FlowFile is used for the Content-type header in responses. Response body content can be cleared by sending an empty (size 0) "
+      "FlowFile for a given URI mapping.";
+
+  EXTENSIONAPI static const core::Property BasePath;
+  EXTENSIONAPI static const core::Property Port;
+  EXTENSIONAPI static const core::Property AuthorizedDNPattern;
+  EXTENSIONAPI static const core::Property SSLCertificate;
+  EXTENSIONAPI static const core::Property SSLCertificateAuthority;
+  EXTENSIONAPI static const core::Property SSLVerifyPeer;
+  EXTENSIONAPI static const core::Property SSLMinimumVersion;
+  EXTENSIONAPI static const core::Property HeadersAsAttributesRegex;
+  EXTENSIONAPI static const core::Property BatchSize;
+  EXTENSIONAPI static const core::Property BufferSize;
+  static auto properties() {
+    return std::array{
+      BasePath,
+      Port,
+      AuthorizedDNPattern,
+      SSLCertificate,
+      SSLCertificateAuthority,
+      SSLVerifyPeer,
+      SSLMinimumVersion,
+      HeadersAsAttributesRegex,
+      BatchSize,
+      BufferSize
+    };
+  }
+
+  EXTENSIONAPI static const core::Relationship Success;
+  static auto relationships() { return std::array{Success}; }
+
+  EXTENSIONAPI static constexpr bool SupportsDynamicProperties = false;
+  EXTENSIONAPI static constexpr bool SupportsDynamicRelationships = false;
+  EXTENSIONAPI static constexpr core::annotation::Input InputRequirement = core::annotation::Input::INPUT_FORBIDDEN;
+  EXTENSIONAPI static constexpr bool IsSingleThreaded = false;
+
+  ADD_COMMON_VIRTUAL_FUNCTIONS_FOR_PROCESSORS
 
   void onTrigger(core::ProcessContext *context, core::ProcessSession *session) override;
   void initialize() override;
@@ -168,10 +189,6 @@ class ListenHTTP : public core::Processor {
  private:
   static const uint64_t DEFAULT_BUFFER_SIZE;
 
-  core::annotation::Input getInputRequirement() const override {
-    return core::annotation::Input::INPUT_FORBIDDEN;
-  }
-
   void processIncomingFlowFile(core::ProcessSession *session);
   void processRequestBuffer(core::ProcessSession *session);
 
diff --git a/extensions/civetweb/protocols/RESTReceiver.cpp b/extensions/civetweb/protocols/RESTReceiver.cpp
index 9523eb8d1..4cb544dfb 100644
--- a/extensions/civetweb/protocols/RESTReceiver.cpp
+++ b/extensions/civetweb/protocols/RESTReceiver.cpp
@@ -1,5 +1,4 @@
 /**
- *
  * 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.
@@ -28,11 +27,7 @@
 #include "core/Resource.h"
 #include "properties/Configuration.h"
 
-namespace org {
-namespace apache {
-namespace nifi {
-namespace minifi {
-namespace c2 {
+namespace org::apache::nifi::minifi::c2 {
 
 int log_message(const struct mg_connection* /*conn*/, const char *message) {
   puts(message);
@@ -111,10 +106,6 @@ std::unique_ptr<CivetServer> RESTReceiver::start_webserver(const std::string &po
   return server;
 }
 
-REGISTER_RESOURCE(RESTReceiver, "Provides a webserver to display C2 heartbeat information");
+REGISTER_RESOURCE(RESTReceiver, DescriptionOnly);
 
-} /* namespace c2 */
-} /* namespace minifi */
-} /* namespace nifi */
-} /* namespace apache */
-} /* namespace org */
+}  // namespace org::apache::nifi::minifi::c2
diff --git a/extensions/civetweb/protocols/RESTReceiver.h b/extensions/civetweb/protocols/RESTReceiver.h
index 349f0215a..b62ac19ed 100644
--- a/extensions/civetweb/protocols/RESTReceiver.h
+++ b/extensions/civetweb/protocols/RESTReceiver.h
@@ -1,5 +1,4 @@
 /**
- *
  * 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.
@@ -24,11 +23,7 @@
 #include "CivetServer.h"
 #include "c2/C2Protocol.h"
 
-namespace org {
-namespace apache {
-namespace nifi {
-namespace minifi {
-namespace c2 {
+namespace org::apache::nifi::minifi::c2 {
 
 int log_message(const struct mg_connection *conn, const char *message);
 
@@ -46,6 +41,8 @@ class RESTReceiver : public RESTProtocol, public HeartbeatReporter {
  public:
   explicit RESTReceiver(const std::string& name, const utils::Identifier& uuid = {});
 
+  EXTENSIONAPI static constexpr const char* Description = "Provides a webserver to display C2 heartbeat information";
+
   void initialize(core::controller::ControllerServiceProvider* controller, state::StateMonitor* updateSink,
                           const std::shared_ptr<Configure> &configure) override;
   int16_t heartbeat(const C2Payload &heartbeat) override;
@@ -91,8 +88,4 @@ class RESTReceiver : public RESTProtocol, public HeartbeatReporter {
   std::shared_ptr<core::logging::Logger> logger_ = core::logging::LoggerFactory<RESTReceiver>::getLogger();
 };
 
-} /* namespace c2 */
-} /* namespace minifi */
-} /* namespace nifi */
-} /* namespace apache */
-} /* namespace org */
+}  // namespace org::apache::nifi::minifi::c2
diff --git a/extensions/coap/controllerservice/CoapConnector.cpp b/extensions/coap/controllerservice/CoapConnector.cpp
index 3840a6a13..e67a359f3 100644
--- a/extensions/coap/controllerservice/CoapConnector.cpp
+++ b/extensions/coap/controllerservice/CoapConnector.cpp
@@ -1,5 +1,4 @@
 /**
- *
  * 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.
@@ -20,30 +19,20 @@
 
 #include <string>
 #include <memory>
-#include <set>
 
 #include "core/logging/LoggerConfiguration.h"
 #include "core/controller/ControllerService.h"
-#include "core/Property.h"
+#include "core/PropertyBuilder.h"
 #include "core/Resource.h"
 #include "io/validation.h"
 #include "properties/Configure.h"
 
-namespace org {
-namespace apache {
-namespace nifi {
-namespace minifi {
-namespace coap {
-namespace controllers {
-
-static core::Property RemoteServer;
-static core::Property Port;
-static core::Property MaxQueueSize;
+namespace org::apache::nifi::minifi::coap::controllers {
 
-core::Property CoapConnectorService::RemoteServer(core::PropertyBuilder::createProperty("Remote Server")->withDescription("Remote CoAP server")->isRequired(false)->build());
-core::Property CoapConnectorService::Port(
+const core::Property CoapConnectorService::RemoteServer(core::PropertyBuilder::createProperty("Remote Server")->withDescription("Remote CoAP server")->isRequired(false)->build());
+const core::Property CoapConnectorService::Port(
     core::PropertyBuilder::createProperty("Remote Port")->withDescription("Remote CoAP server port")->withDefaultValue<uint64_t>(8181)->isRequired(true)->build());
-core::Property CoapConnectorService::MaxQueueSize(
+const core::Property CoapConnectorService::MaxQueueSize(
     core::PropertyBuilder::createProperty("Max Queue Size")->withDescription("Max queue size for received data ")->withDefaultValue<uint64_t>(1000)->isRequired(false)->build());
 
 void CoapConnectorService::initialize() {
@@ -84,18 +73,9 @@ CoapResponse CoapConnectorService::sendPayload(uint8_t type, const std::string &
 }
 
 void CoapConnectorService::initializeProperties() {
-  std::set<core::Property> supportedProperties;
-  supportedProperties.insert(RemoteServer);
-  supportedProperties.insert(Port);
-  supportedProperties.insert(MaxQueueSize);
-  setSupportedProperties(supportedProperties);
+  setSupportedProperties(properties());
 }
 
-REGISTER_INTERNAL_RESOURCE(CoapConnectorService);
+REGISTER_RESOURCE(CoapConnectorService, InternalResource);
 
-} /* namespace controllers */
-} /* namespace coap */
-} /* namespace minifi */
-} /* namespace nifi */
-} /* namespace apache */
-} /* namespace org */
+}  // namespace org::apache::nifi::minifi::coap::controllers
diff --git a/extensions/coap/controllerservice/CoapConnector.h b/extensions/coap/controllerservice/CoapConnector.h
index cba303e58..f8ecb6ae5 100644
--- a/extensions/coap/controllerservice/CoapConnector.h
+++ b/extensions/coap/controllerservice/CoapConnector.h
@@ -1,5 +1,4 @@
 /**
- *
  * 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.
@@ -32,12 +31,7 @@
 #include "coap_connection.h"
 #include "coap_message.h"
 
-namespace org {
-namespace apache {
-namespace nifi {
-namespace minifi {
-namespace coap {
-namespace controllers {
+namespace org::apache::nifi::minifi::coap::controllers {
 
 /**
  * Purpose and Justification: Controller services function as a layerable way to provide
@@ -57,12 +51,18 @@ class CoapConnectorService : public core::controller::ControllerService {
     initialize();
   }
 
-  /**
-   * Parameters needed.
-   */
-  static core::Property RemoteServer;
-  static core::Property Port;
-  static core::Property MaxQueueSize;
+  EXTENSIONAPI static const core::Property RemoteServer;
+  EXTENSIONAPI static const core::Property Port;
+  EXTENSIONAPI static const core::Property MaxQueueSize;
+  static auto properties() {
+    return std::array{
+      RemoteServer,
+      Port,
+      MaxQueueSize
+    };
+  }
+  EXTENSIONAPI static constexpr bool SupportsDynamicProperties = false;
+  ADD_COMMON_VIRTUAL_FUNCTIONS_FOR_CONTROLLER_SERVICES
 
   void initialize() override;
 
@@ -108,9 +108,4 @@ class CoapConnectorService : public core::controller::ControllerService {
   std::shared_ptr<core::logging::Logger> logger_{ core::logging::LoggerFactory<CoapConnectorService>::getLogger() };
 };
 
-} /* namespace controllers */
-} /* namespace coap */
-} /* namespace minifi */
-} /* namespace nifi */
-} /* namespace apache */
-} /* namespace org */
+}  // namespace org::apache::nifi::minifi::coap::controllers
diff --git a/extensions/coap/protocols/CoapC2Protocol.cpp b/extensions/coap/protocols/CoapC2Protocol.cpp
index 04e6df9cc..e6210d4c0 100644
--- a/extensions/coap/protocols/CoapC2Protocol.cpp
+++ b/extensions/coap/protocols/CoapC2Protocol.cpp
@@ -1,5 +1,4 @@
 /**
- *
  * 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.
@@ -23,12 +22,7 @@
 #include "io/BaseStream.h"
 #include "core/Resource.h"
 
-namespace org {
-namespace apache {
-namespace nifi {
-namespace minifi {
-namespace coap {
-namespace c2 {
+namespace org::apache::nifi::minifi::coap::c2 {
 
 uint8_t CoapProtocol::REGISTRATION_MSG[8] = { 0x72, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72 };
 
@@ -292,11 +286,6 @@ minifi::c2::C2Payload CoapProtocol::serialize(const minifi::c2::C2Payload &paylo
   return minifi::c2::C2Payload(payload.getOperation(), state::UpdateState::READ_ERROR);
 }
 
-REGISTER_INTERNAL_RESOURCE(CoapProtocol);
+REGISTER_RESOURCE(CoapProtocol, InternalResource);
 
-} /* namespace c2 */
-} /* namespace coap */
-} /* namespace minifi */
-} /* namespace nifi */
-} /* namespace apache */
-} /* namespace org */
+}  // namespace org::apache::nifi::minifi::coap::c2
diff --git a/extensions/coap/protocols/CoapC2Protocol.h b/extensions/coap/protocols/CoapC2Protocol.h
index 5aae5e28b..fe5420a69 100644
--- a/extensions/coap/protocols/CoapC2Protocol.h
+++ b/extensions/coap/protocols/CoapC2Protocol.h
@@ -1,5 +1,4 @@
 /**
- *
  * 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.
@@ -38,12 +37,7 @@
 #include "coap2/address.h"
 #include "protocols/RESTSender.h"
 
-namespace org {
-namespace apache {
-namespace nifi {
-namespace minifi {
-namespace coap {
-namespace c2 {
+namespace org::apache::nifi::minifi::coap::c2 {
 
 #define REQUIRE_VALID(x) \
   if (io::isError(x)) { \
@@ -65,12 +59,10 @@ class CoapProtocol : public minifi::c2::RESTSender {
 
   ~CoapProtocol() override;
 
-  /**
-   * Consume the payload.
-   * @param url to evaluate.
-   * @param payload payload to consume.
-   * @param direction direction of operation.
-   */
+  static auto properties() { return std::array<core::Property, 0>{}; }
+  EXTENSIONAPI static constexpr bool SupportsDynamicProperties = false;
+  EXTENSIONAPI static constexpr bool SupportsDynamicRelationships = false;
+
   minifi::c2::C2Payload consumePayload(const std::string &url, const minifi::c2::C2Payload &payload, minifi::c2::Direction direction, bool async) override;
 
   minifi::c2::C2Payload consumePayload(const minifi::c2::C2Payload &payload, minifi::c2::Direction /*direction*/, bool /*async*/) override {
@@ -83,8 +75,6 @@ class CoapProtocol : public minifi::c2::RESTSender {
 
   void initialize(core::controller::ControllerServiceProvider* controller, const std::shared_ptr<Configure> &configure) override;
 
-  // Supported Properties
-
  protected:
   static bool isRegistrationMessage(controllers::CoapResponse &response) {
     const auto response_data = response.getData();
@@ -133,9 +123,4 @@ class CoapProtocol : public minifi::c2::RESTSender {
   std::shared_ptr<core::logging::Logger> logger_ = core::logging::LoggerFactory<CoapProtocol>::getLogger();
 };
 
-} /* namespace c2 */
-} /* namespace coap */
-} /* namespace minifi */
-} /* namespace nifi */
-} /* namespace apache */
-} /* namespace org */
+}  // namespace org::apache::nifi::minifi::coap::c2
diff --git a/extensions/expression-language/ExpressionContextBuilder.cpp b/extensions/expression-language/ExpressionContextBuilder.cpp
index 5a568e75c..165da6cff 100644
--- a/extensions/expression-language/ExpressionContextBuilder.cpp
+++ b/extensions/expression-language/ExpressionContextBuilder.cpp
@@ -23,12 +23,7 @@
 #include "ProcessContextExpr.h"
 #include "core/Resource.h"
 
-namespace org {
-namespace apache {
-namespace nifi {
-namespace minifi {
-namespace core {
-namespace expressions {
+namespace org::apache::nifi::minifi::core::expressions {
 
 ExpressionContextBuilder::ExpressionContextBuilder(const std::string &name, const minifi::utils::Identifier &uuid)
     : core::ProcessContextBuilder(name, uuid) {
@@ -44,11 +39,6 @@ std::shared_ptr<core::ProcessContext> ExpressionContextBuilder::build(const std:
   return std::make_shared<core::ProcessContextExpr>(processor, controller_service_provider_, prov_repo_, flow_repo_, configuration_, content_repo_);
 }
 
-REGISTER_INTERNAL_RESOURCE_AS(ExpressionContextBuilder, ("ProcessContextBuilder"));
+REGISTER_RESOURCE_AS(ExpressionContextBuilder, InternalResource, ("ProcessContextBuilder"));
 
-} /* namespace expressions */
-} /* namespace core */
-} /* namespace minifi */
-} /* namespace nifi */
-} /* namespace apache */
-} /* namespace org */
+}  // namespace org::apache::nifi::minifi::core::expressions
diff --git a/extensions/expression-language/ExpressionContextBuilder.h b/extensions/expression-language/ExpressionContextBuilder.h
index ea2068194..66cb215b5 100644
--- a/extensions/expression-language/ExpressionContextBuilder.h
+++ b/extensions/expression-language/ExpressionContextBuilder.h
@@ -15,20 +15,14 @@
  * limitations under the License.
  */
 
-#ifndef EXTENSIONS_EXPRESSION_LANGUAGE_EXPRESSIONCONTEXTBUILDER_H_
-#define EXTENSIONS_EXPRESSION_LANGUAGE_EXPRESSIONCONTEXTBUILDER_H_
+#pragma once
 
 #include <string>
 #include <memory>
 
 #include "core/ProcessContextBuilder.h"
 
-namespace org {
-namespace apache {
-namespace nifi {
-namespace minifi {
-namespace core {
-namespace expressions {
+namespace org::apache::nifi::minifi::core::expressions {
 
 /**
  *   Purpose: Creates a context builder that can be used by the class loader to inject EL functionality
@@ -45,14 +39,11 @@ class ExpressionContextBuilder : public core::ProcessContextBuilder {
 
   virtual ~ExpressionContextBuilder();
 
+  static auto properties() { return std::array<core::Property, 0>{}; }
+  EXTENSIONAPI static constexpr bool SupportsDynamicProperties = false;
+  EXTENSIONAPI static constexpr bool SupportsDynamicRelationships = false;
+
   std::shared_ptr<core::ProcessContext> build(const std::shared_ptr<ProcessorNode> &processor) override;
 };
 
-} /* namespace expressions */
-} /* namespace core */
-} /* namespace minifi */
-} /* namespace nifi */
-} /* namespace apache */
-} /* namespace org */
-
-#endif /* EXTENSIONS_EXPRESSION_LANGUAGE_EXPRESSIONCONTEXTBUILDER_H_ */
+}  // namespace org::apache::nifi::minifi::core::expressions
diff --git a/extensions/expression-language/tests/ProcessContextExprTests.cpp b/extensions/expression-language/tests/ProcessContextExprTests.cpp
index 1cef4eabc..50ca51e82 100644
--- a/extensions/expression-language/tests/ProcessContextExprTests.cpp
+++ b/extensions/expression-language/tests/ProcessContextExprTests.cpp
@@ -22,7 +22,7 @@
 #include "core/Resource.h"
 #include "ProcessContextExpr.h"
 #include "Processor.h"
-#include "Property.h"
+#include "PropertyBuilder.h"
 #include "TestBase.h"
 #include "Catch.h"
 
@@ -31,24 +31,33 @@ namespace org::apache::nifi::minifi {
 class DummyProcessor : public core::Processor {
  public:
   using core::Processor::Processor;
-  static core::Property SimpleProperty;
-  static core::Property ExpressionLanguageProperty;
-  void initialize() override { setSupportedProperties({SimpleProperty, ExpressionLanguageProperty}); }
-  bool supportsDynamicProperties() override { return true; }
+
+  static constexpr const char* Description = "A processor that does nothing.";
+  static const core::Property SimpleProperty;
+  static const core::Property ExpressionLanguageProperty;
+  static auto properties() { return std::array{SimpleProperty, ExpressionLanguageProperty}; }
+  static auto relationships() { return std::array<core::Relationship, 0>{}; }
+  static constexpr bool SupportsDynamicProperties = true;
+  static constexpr bool SupportsDynamicRelationships = true;
+  static constexpr core::annotation::Input InputRequirement = core::annotation::Input::INPUT_ALLOWED;
+  static constexpr bool IsSingleThreaded = false;
+  ADD_COMMON_VIRTUAL_FUNCTIONS_FOR_PROCESSORS
+
+  void initialize() override { setSupportedProperties(properties()); }
 };
 
-core::Property DummyProcessor::SimpleProperty{
+const core::Property DummyProcessor::SimpleProperty{
     core::PropertyBuilder::createProperty("Simple Property")
         ->withDescription("Just a simple string property")
         ->build()};
 
-core::Property DummyProcessor::ExpressionLanguageProperty{
+const core::Property DummyProcessor::ExpressionLanguageProperty{
     core::PropertyBuilder::createProperty("Expression Language Property")
         ->withDescription("A property which supports expression language")
         ->supportsExpressionLanguage(true)
         ->build()};
 
-REGISTER_RESOURCE(DummyProcessor, "A processor that does nothing.");
+REGISTER_RESOURCE(DummyProcessor, Processor);
 
 }  // namespace org::apache::nifi::minifi
 
diff --git a/extensions/expression-language/tests/RouteOnAttributeTests.cpp b/extensions/expression-language/tests/RouteOnAttributeTests.cpp
index 78d3657bf..863847149 100644
--- a/extensions/expression-language/tests/RouteOnAttributeTests.cpp
+++ b/extensions/expression-language/tests/RouteOnAttributeTests.cpp
@@ -15,6 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+#include <array>
 #include <memory>
 #include "TestBase.h"
 #include "Catch.h"
@@ -38,7 +39,7 @@ TEST_CASE("RouteOnAttributeMatchedTest", "[routeOnAttributeMatchedTest]") {
   plan->setProperty(update_proc, "route_condition_attr", "true", true);
 
   const auto &route_proc = plan->addProcessor("RouteOnAttribute", "route", core::Relationship("success", "description"), true);
-  route_proc->setAutoTerminatedRelationships({ { core::Relationship("unmatched", "description") } });
+  route_proc->setAutoTerminatedRelationships(std::array{core::Relationship("unmatched", "description")});
   plan->setProperty(route_proc, "route_matched", "${route_condition_attr}", true);
 
   const auto &update_matched_proc = plan->addProcessor("UpdateAttribute", "update_matched", core::Relationship("route_matched", "description"), true);
diff --git a/extensions/gcp/controllerservices/GCPCredentialsControllerService.cpp b/extensions/gcp/controllerservices/GCPCredentialsControllerService.cpp
index 860a43afd..167b13f8e 100644
--- a/extensions/gcp/controllerservices/GCPCredentialsControllerService.cpp
+++ b/extensions/gcp/controllerservices/GCPCredentialsControllerService.cpp
@@ -18,6 +18,7 @@
 
 #include "GCPCredentialsControllerService.h"
 
+#include "core/PropertyBuilder.h"
 #include "core/Resource.h"
 #include "google/cloud/storage/client.h"
 
@@ -46,7 +47,7 @@ const core::Property GCPCredentialsControllerService::JsonContents(
         ->build());
 
 void GCPCredentialsControllerService::initialize() {
-  setSupportedProperties({CredentialsLoc, JsonFilePath, JsonContents});
+  setSupportedProperties(properties());
 }
 
 std::shared_ptr<gcs::oauth2::Credentials> GCPCredentialsControllerService::createDefaultCredentials() const {
@@ -109,6 +110,6 @@ void GCPCredentialsControllerService::onEnable() {
     logger_->log_error("Couldn't create valid credentials");
 }
 
-REGISTER_RESOURCE(GCPCredentialsControllerService, "Google Cloud Platform Credentials Controller Service");
+REGISTER_RESOURCE(GCPCredentialsControllerService, ControllerService);
 }  // namespace org::apache::nifi::minifi::extensions::gcp
 
diff --git a/extensions/gcp/controllerservices/GCPCredentialsControllerService.h b/extensions/gcp/controllerservices/GCPCredentialsControllerService.h
index cc7a628c4..a3d1b30ca 100644
--- a/extensions/gcp/controllerservices/GCPCredentialsControllerService.h
+++ b/extensions/gcp/controllerservices/GCPCredentialsControllerService.h
@@ -38,9 +38,21 @@ class GCPCredentialsControllerService : public core::controller::ControllerServi
              (USE_JSON_CONTENTS, "Service Account JSON"),
              (USE_ANONYMOUS_CREDENTIALS, "Use Anonymous credentials"));
 
+  EXTENSIONAPI static constexpr const char* Description = "Google Cloud Platform Credentials Controller Service";
+
   EXTENSIONAPI static const core::Property CredentialsLoc;
   EXTENSIONAPI static const core::Property JsonFilePath;
   EXTENSIONAPI static const core::Property JsonContents;
+  static auto properties() {
+    return std::array{
+      CredentialsLoc,
+      JsonFilePath,
+      JsonContents
+    };
+  }
+
+  EXTENSIONAPI static constexpr bool SupportsDynamicProperties = false;
+  ADD_COMMON_VIRTUAL_FUNCTIONS_FOR_CONTROLLER_SERVICES
 
   using ControllerService::ControllerService;
 
diff --git a/extensions/gcp/processors/DeleteGCSObject.cpp b/extensions/gcp/processors/DeleteGCSObject.cpp
index e95178df1..03da92993 100644
--- a/extensions/gcp/processors/DeleteGCSObject.cpp
+++ b/extensions/gcp/processors/DeleteGCSObject.cpp
@@ -26,45 +26,9 @@
 namespace gcs = ::google::cloud::storage;
 
 namespace org::apache::nifi::minifi::extensions::gcp {
-const core::Property DeleteGCSObject::Bucket(
-    core::PropertyBuilder::createProperty("Bucket")
-        ->withDescription("Bucket of the object.")
-        ->withDefaultValue("${gcs.bucket}")
-        ->supportsExpressionLanguage(true)
-        ->build());
-
-const core::Property DeleteGCSObject::Key(
-    core::PropertyBuilder::createProperty("Key")
-        ->withDescription("Name of the object.")
-        ->withDefaultValue("${filename}")
-        ->supportsExpressionLanguage(true)
-        ->build());
-
-const core::Property DeleteGCSObject::ObjectGeneration(
-    core::PropertyBuilder::createProperty("Object Generation")
-        ->withDescription("The generation of the Object to download. If left empty, then it will download the latest generation.")
-        ->supportsExpressionLanguage(true)
-        ->build());
-
-const core::Property DeleteGCSObject::EncryptionKey(
-    core::PropertyBuilder::createProperty("Server Side Encryption Key")
-        ->withDescription("The AES256 Encryption Key (encoded in base64) for server-side decryption of the object.")
-        ->isRequired(false)
-        ->supportsExpressionLanguage(true)
-        ->build());
-
-const core::Relationship DeleteGCSObject::Success("success", "FlowFiles are routed to this relationship after a successful Google Cloud Storage operation.");
-const core::Relationship DeleteGCSObject::Failure("failure", "FlowFiles are routed to this relationship if the Google Cloud Storage operation fails.");
-
 void DeleteGCSObject::initialize() {
-  setSupportedProperties({GCPCredentials,
-                          Bucket,
-                          Key,
-                          ObjectGeneration,
-                          NumberOfRetries,
-                          EncryptionKey,
-                          EndpointOverrideURL});
-  setSupportedRelationships({Success, Failure});
+  setSupportedProperties(properties());
+  setSupportedRelationships(relationships());
 }
 
 void DeleteGCSObject::onTrigger(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSession>& session) {
@@ -116,6 +80,4 @@ void DeleteGCSObject::onTrigger(const std::shared_ptr<core::ProcessContext>& con
 
   session->transfer(flow_file, Success);
 }
-
-REGISTER_RESOURCE(DeleteGCSObject, "Deletes an object from a Google Cloud Bucket.");
 }  // namespace org::apache::nifi::minifi::extensions::gcp
diff --git a/extensions/gcp/processors/DeleteGCSObject.h b/extensions/gcp/processors/DeleteGCSObject.h
index 22644c9a9..ae263a978 100644
--- a/extensions/gcp/processors/DeleteGCSObject.h
+++ b/extensions/gcp/processors/DeleteGCSObject.h
@@ -22,6 +22,7 @@
 
 #include "GCSProcessor.h"
 #include "core/logging/LoggerConfiguration.h"
+#include "utils/ArrayUtils.h"
 
 namespace org::apache::nifi::minifi::extensions::gcp {
 
@@ -32,24 +33,34 @@ class DeleteGCSObject : public GCSProcessor {
   }
   ~DeleteGCSObject() override = default;
 
+  EXTENSIONAPI static constexpr const char* Description = "Deletes an object from a Google Cloud Bucket.";
+
   EXTENSIONAPI static const core::Property Bucket;
   EXTENSIONAPI static const core::Property Key;
   EXTENSIONAPI static const core::Property EncryptionKey;
   EXTENSIONAPI static const core::Property ObjectGeneration;
+  static auto properties() {
+    return utils::array_cat(GCSProcessor::properties(), std::array{
+      Bucket,
+      Key,
+      EncryptionKey,
+      ObjectGeneration
+    });
+  }
 
   EXTENSIONAPI static const core::Relationship Success;
   EXTENSIONAPI static const core::Relationship Failure;
+  static auto relationships() { return std::array{Success, Failure}; }
 
-  void initialize() override;
-  void onTrigger(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSession>& session) override;
+  EXTENSIONAPI static constexpr bool SupportsDynamicProperties = false;
+  EXTENSIONAPI static constexpr bool SupportsDynamicRelationships = false;
+  EXTENSIONAPI static constexpr core::annotation::Input InputRequirement = core::annotation::Input::INPUT_REQUIRED;
+  EXTENSIONAPI static constexpr bool IsSingleThreaded = true;
 
-  core::annotation::Input getInputRequirement() const override {
-    return core::annotation::Input::INPUT_REQUIRED;
-  }
+  ADD_COMMON_VIRTUAL_FUNCTIONS_FOR_PROCESSORS
 
-  bool isSingleThreaded() const override {
-    return true;
-  }
+  void initialize() override;
+  void onTrigger(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSession>& session) override;
 };
 
 }  // namespace org::apache::nifi::minifi::extensions::gcp
diff --git a/extensions/gcp/processors/FetchGCSObject.cpp b/extensions/gcp/processors/FetchGCSObject.cpp
index 58e36df31..33f4b85d9 100644
--- a/extensions/gcp/processors/FetchGCSObject.cpp
+++ b/extensions/gcp/processors/FetchGCSObject.cpp
@@ -28,37 +28,6 @@
 namespace gcs = ::google::cloud::storage;
 
 namespace org::apache::nifi::minifi::extensions::gcp {
-const core::Property FetchGCSObject::Bucket(
-    core::PropertyBuilder::createProperty("Bucket")
-        ->withDescription("Bucket of the object.")
-        ->withDefaultValue("${gcs.bucket}")
-        ->supportsExpressionLanguage(true)
-        ->build());
-
-const core::Property FetchGCSObject::Key(
-    core::PropertyBuilder::createProperty("Key")
-        ->withDescription("Name of the object.")
-        ->withDefaultValue("${filename}")
-        ->supportsExpressionLanguage(true)
-        ->build());
-
-const core::Property FetchGCSObject::ObjectGeneration(
-    core::PropertyBuilder::createProperty("Object Generation")
-        ->withDescription("The generation of the Object to download. If left empty, then it will download the latest generation.")
-        ->supportsExpressionLanguage(true)
-        ->build());
-
-const core::Property FetchGCSObject::EncryptionKey(
-    core::PropertyBuilder::createProperty("Server Side Encryption Key")
-        ->withDescription("The AES256 Encryption Key (encoded in base64) for server-side decryption of the object.")
-        ->isRequired(false)
-        ->supportsExpressionLanguage(true)
-        ->build());
-
-const core::Relationship FetchGCSObject::Success("success", "FlowFiles are routed to this relationship after a successful Google Cloud Storage operation.");
-const core::Relationship FetchGCSObject::Failure("failure", "FlowFiles are routed to this relationship if the Google Cloud Storage operation fails.");
-
-
 namespace {
 class FetchFromGCSCallback {
  public:
@@ -115,14 +84,8 @@ class FetchFromGCSCallback {
 
 
 void FetchGCSObject::initialize() {
-  setSupportedProperties({GCPCredentials,
-                          Bucket,
-                          Key,
-                          ObjectGeneration,
-                          NumberOfRetries,
-                          EncryptionKey,
-                          EndpointOverrideURL});
-  setSupportedRelationships({Success, Failure});
+  setSupportedProperties(properties());
+  setSupportedRelationships(relationships());
 }
 
 void FetchGCSObject::onSchedule(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSessionFactory>& session_factory) {
@@ -192,6 +155,4 @@ void FetchGCSObject::onTrigger(const std::shared_ptr<core::ProcessContext>& cont
     flow_file->setAttribute(GCS_STORAGE_CLASS, *storage_class);
   session->transfer(flow_file, Success);
 }
-
-REGISTER_RESOURCE(FetchGCSObject, "Fetches a file from a Google Cloud Bucket. Designed to be used in tandem with ListGCSBucket.");
 }  // namespace org::apache::nifi::minifi::extensions::gcp
diff --git a/extensions/gcp/processors/FetchGCSObject.h b/extensions/gcp/processors/FetchGCSObject.h
index d71e747e5..794e7b4cd 100644
--- a/extensions/gcp/processors/FetchGCSObject.h
+++ b/extensions/gcp/processors/FetchGCSObject.h
@@ -23,6 +23,7 @@
 #include "GCSProcessor.h"
 #include "google/cloud/storage/well_known_headers.h"
 #include "core/logging/LoggerConfiguration.h"
+#include "utils/ArrayUtils.h"
 
 namespace org::apache::nifi::minifi::extensions::gcp {
 
@@ -33,26 +34,36 @@ class FetchGCSObject : public GCSProcessor {
   }
   ~FetchGCSObject() override = default;
 
+  EXTENSIONAPI static constexpr const char* Description = "Fetches a file from a Google Cloud Bucket. Designed to be used in tandem with ListGCSBucket.";
+
   EXTENSIONAPI static const core::Property Bucket;
   EXTENSIONAPI static const core::Property Key;
   EXTENSIONAPI static const core::Property EncryptionKey;
   EXTENSIONAPI static const core::Property ObjectGeneration;
+  static auto properties() {
+    return utils::array_cat(GCSProcessor::properties(), std::array{
+      Bucket,
+      Key,
+      EncryptionKey,
+      ObjectGeneration
+    });
+  }
 
   EXTENSIONAPI static const core::Relationship Success;
   EXTENSIONAPI static const core::Relationship Failure;
+  static auto relationships() { return std::array{Success, Failure}; }
+
+  EXTENSIONAPI static constexpr bool SupportsDynamicProperties = false;
+  EXTENSIONAPI static constexpr bool SupportsDynamicRelationships = false;
+  EXTENSIONAPI static constexpr core::annotation::Input InputRequirement = core::annotation::Input::INPUT_REQUIRED;
+  EXTENSIONAPI static constexpr bool IsSingleThreaded = true;
+
+  ADD_COMMON_VIRTUAL_FUNCTIONS_FOR_PROCESSORS
 
   void initialize() override;
   void onSchedule(const std::shared_ptr<core::ProcessContext> &context, const std::shared_ptr<core::ProcessSessionFactory> &sessionFactory) override;
   void onTrigger(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSession>& session) override;
 
-  core::annotation::Input getInputRequirement() const override {
-    return core::annotation::Input::INPUT_REQUIRED;
-  }
-
-  bool isSingleThreaded() const override {
-    return true;
-  }
-
  private:
   google::cloud::storage::EncryptionKey encryption_key_;
 };
diff --git a/extensions/gcp/processors/GCSProcessor.cpp b/extensions/gcp/processors/GCSProcessor.cpp
index 445f2959e..181461af4 100644
--- a/extensions/gcp/processors/GCSProcessor.cpp
+++ b/extensions/gcp/processors/GCSProcessor.cpp
@@ -24,28 +24,6 @@ namespace gcs = ::google::cloud::storage;
 
 namespace org::apache::nifi::minifi::extensions::gcp {
 
-const core::Property GCSProcessor::GCPCredentials(
-    core::PropertyBuilder::createProperty("GCP Credentials Provider Service")
-        ->withDescription("The Controller Service used to obtain Google Cloud Platform credentials.")
-        ->isRequired(true)
-        ->asType<GCPCredentialsControllerService>()
-        ->build());
-
-const core::Property GCSProcessor::NumberOfRetries(
-    core::PropertyBuilder::createProperty("Number of retries")
-        ->withDescription("How many retry attempts should be made before routing to the failure relationship.")
-        ->withDefaultValue<uint64_t>(6)
-        ->isRequired(true)
-        ->supportsExpressionLanguage(false)
-        ->build());
-
-const core::Property GCSProcessor::EndpointOverrideURL(
-    core::PropertyBuilder::createProperty("Endpoint Override URL")
-        ->withDescription("Overrides the default Google Cloud Storage endpoints")
-        ->isRequired(false)
-        ->supportsExpressionLanguage(true)
-        ->build());
-
 namespace {
 std::shared_ptr<google::cloud::storage::oauth2::Credentials> getCredentials(core::ProcessContext& context) {
   std::string service_name;
@@ -59,7 +37,6 @@ std::shared_ptr<google::cloud::storage::oauth2::Credentials> getCredentials(core
 }
 }  // namespace
 
-
 void GCSProcessor::onSchedule(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSessionFactory>&) {
   gsl_Expects(context);
   if (auto number_of_retries = context->getProperty<uint64_t>(NumberOfRetries)) {
diff --git a/extensions/gcp/processors/GCSProcessor.h b/extensions/gcp/processors/GCSProcessor.h
index f673a3c7d..78fcdd51e 100644
--- a/extensions/gcp/processors/GCSProcessor.h
+++ b/extensions/gcp/processors/GCSProcessor.h
@@ -40,6 +40,13 @@ class GCSProcessor : public core::Processor {
   EXTENSIONAPI static const core::Property GCPCredentials;
   EXTENSIONAPI static const core::Property NumberOfRetries;
   EXTENSIONAPI static const core::Property EndpointOverrideURL;
+  static auto properties() {
+    return std::array{
+      GCPCredentials,
+      NumberOfRetries,
+      EndpointOverrideURL
+    };
+  }
 
   void onSchedule(const std::shared_ptr<core::ProcessContext> &context, const std::shared_ptr<core::ProcessSessionFactory> &sessionFactory) override;
 
diff --git a/extensions/gcp/processors/GCSProcessorStaticDefinitions.cpp b/extensions/gcp/processors/GCSProcessorStaticDefinitions.cpp
new file mode 100644
index 000000000..2c5b32b4b
--- /dev/null
+++ b/extensions/gcp/processors/GCSProcessorStaticDefinitions.cpp
@@ -0,0 +1,210 @@
+/**
+ * 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 "DeleteGCSObject.h"
+#include "FetchGCSObject.h"
+#include "GCSProcessor.h"
+#include "ListGCSBucket.h"
+#include "PutGCSObject.h"
+#include "../controllerservices/GCPCredentialsControllerService.h"
+#include "core/PropertyBuilder.h"
+#include "core/Resource.h"
+
+// FIXME(fgerlits): we need to put all these static definitions in a single file so that they are executed in this order at runtime
+// when https://issues.apache.org/jira/browse/MINIFICPP-1825 is closed, these definitions should be moved back to the cpp file of the class to which they belong
+
+namespace org::apache::nifi::minifi::extensions::gcp {
+
+// GCSProcessor
+
+const core::Property GCSProcessor::GCPCredentials(
+    core::PropertyBuilder::createProperty("GCP Credentials Provider Service")
+        ->withDescription("The Controller Service used to obtain Google Cloud Platform credentials.")
+        ->isRequired(true)
+        ->asType<GCPCredentialsControllerService>()
+        ->build());
+
+const core::Property GCSProcessor::NumberOfRetries(
+    core::PropertyBuilder::createProperty("Number of retries")
+        ->withDescription("How many retry attempts should be made before routing to the failure relationship.")
+        ->withDefaultValue<uint64_t>(6)
+        ->isRequired(true)
+        ->supportsExpressionLanguage(false)
+        ->build());
+
+const core::Property GCSProcessor::EndpointOverrideURL(
+    core::PropertyBuilder::createProperty("Endpoint Override URL")
+        ->withDescription("Overrides the default Google Cloud Storage endpoints")
+        ->isRequired(false)
+        ->supportsExpressionLanguage(true)
+        ->build());
+
+
+// DeleteGCSObject
+
+const core::Property DeleteGCSObject::Bucket(
+    core::PropertyBuilder::createProperty("Bucket")
+        ->withDescription("Bucket of the object.")
+        ->withDefaultValue("${gcs.bucket}")
+        ->supportsExpressionLanguage(true)
+        ->build());
+
+const core::Property DeleteGCSObject::Key(
+    core::PropertyBuilder::createProperty("Key")
+        ->withDescription("Name of the object.")
+        ->withDefaultValue("${filename}")
+        ->supportsExpressionLanguage(true)
+        ->build());
+
+const core::Property DeleteGCSObject::ObjectGeneration(
+    core::PropertyBuilder::createProperty("Object Generation")
+        ->withDescription("The generation of the Object to download. If left empty, then it will download the latest generation.")
+        ->supportsExpressionLanguage(true)
+        ->build());
+
+const core::Property DeleteGCSObject::EncryptionKey(
+    core::PropertyBuilder::createProperty("Server Side Encryption Key")
+        ->withDescription("The AES256 Encryption Key (encoded in base64) for server-side decryption of the object.")
+        ->isRequired(false)
+        ->supportsExpressionLanguage(true)
+        ->build());
+
+const core::Relationship DeleteGCSObject::Success("success", "FlowFiles are routed to this relationship after a successful Google Cloud Storage operation.");
+const core::Relationship DeleteGCSObject::Failure("failure", "FlowFiles are routed to this relationship if the Google Cloud Storage operation fails.");
+
+REGISTER_RESOURCE(DeleteGCSObject, Processor);
+
+
+// FetchGCSObject
+
+const core::Property FetchGCSObject::Bucket(
+    core::PropertyBuilder::createProperty("Bucket")
+        ->withDescription("Bucket of the object.")
+        ->withDefaultValue("${gcs.bucket}")
+        ->supportsExpressionLanguage(true)
+        ->build());
+
+const core::Property FetchGCSObject::Key(
+    core::PropertyBuilder::createProperty("Key")
+        ->withDescription("Name of the object.")
+        ->withDefaultValue("${filename}")
+        ->supportsExpressionLanguage(true)
+        ->build());
+
+const core::Property FetchGCSObject::ObjectGeneration(
+    core::PropertyBuilder::createProperty("Object Generation")
+        ->withDescription("The generation of the Object to download. If left empty, then it will download the latest generation.")
+        ->supportsExpressionLanguage(true)
+        ->build());
+
+const core::Property FetchGCSObject::EncryptionKey(
+    core::PropertyBuilder::createProperty("Server Side Encryption Key")
+        ->withDescription("The AES256 Encryption Key (encoded in base64) for server-side decryption of the object.")
+        ->isRequired(false)
+        ->supportsExpressionLanguage(true)
+        ->build());
+
+const core::Relationship FetchGCSObject::Success("success", "FlowFiles are routed to this relationship after a successful Google Cloud Storage operation.");
+const core::Relationship FetchGCSObject::Failure("failure", "FlowFiles are routed to this relationship if the Google Cloud Storage operation fails.");
+
+REGISTER_RESOURCE(FetchGCSObject, Processor);
+
+
+// ListGCSBucket
+
+const core::Property ListGCSBucket::Bucket(
+    core::PropertyBuilder::createProperty("Bucket")
+        ->withDescription("Bucket of the object.")
+        ->isRequired(true)
+        ->supportsExpressionLanguage(true)
+        ->build());
+
+const core::Property ListGCSBucket::ListAllVersions(
+    core::PropertyBuilder::createProperty("List all versions")
+        ->withDescription("Set this option to `true` to get all the previous versions separately.")
+        ->withDefaultValue<bool>(false)
+        ->build());
+
+const core::Relationship ListGCSBucket::Success("success", "FlowFiles are routed to this relationship after a successful Google Cloud Storage operation.");
+
+REGISTER_RESOURCE(ListGCSBucket, Processor);
+
+
+// PutGCSObject
+
+const core::Property PutGCSObject::Bucket(
+    core::PropertyBuilder::createProperty("Bucket")
+        ->withDescription("Bucket of the object.")
+        ->withDefaultValue("${gcs.bucket}")
+        ->supportsExpressionLanguage(true)
+        ->build());
+
+const core::Property PutGCSObject::Key(
+    core::PropertyBuilder::createProperty("Key")
+        ->withDescription("Name of the object.")
+        ->withDefaultValue("${filename}")
+        ->supportsExpressionLanguage(true)
+        ->build());
+
+const core::Property PutGCSObject::ContentType(
+    core::PropertyBuilder::createProperty("Content Type")
+        ->withDescription("Content Type for the file, i.e. text/plain ")
+        ->isRequired(false)
+        ->withDefaultValue("${mime.type}")
+        ->supportsExpressionLanguage(true)
+        ->build());
+
+const core::Property PutGCSObject::MD5Hash(
+    core::PropertyBuilder::createProperty("MD5 Hash")
+        ->withDescription("MD5 Hash (encoded in Base64) of the file for server-side validation.")
+        ->isRequired(false)
+        ->supportsExpressionLanguage(true)
+        ->build());
+
+const core::Property PutGCSObject::Crc32cChecksum(
+    core::PropertyBuilder::createProperty("CRC32C Checksum")
+        ->withDescription("CRC32C Checksum (encoded in Base64, big-Endian order) of the file for server-side validation.")
+        ->isRequired(false)
+        ->supportsExpressionLanguage(true)
+        ->build());
+
+const core::Property PutGCSObject::EncryptionKey(
+    core::PropertyBuilder::createProperty("Server Side Encryption Key")
+        ->withDescription("An AES256 Encryption Key (encoded in base64) for server-side encryption of the object.")
+        ->isRequired(false)
+        ->supportsExpressionLanguage(true)
+        ->build());
+
+const core::Property PutGCSObject::ObjectACL(
+    core::PropertyBuilder::createProperty("Object ACL")
+        ->withDescription("Access Control to be attached to the object uploaded. Not providing this will revert to bucket defaults.")
+        ->isRequired(false)
+        ->withAllowableValues(PredefinedAcl::values())
+        ->build());
+
+const core::Property PutGCSObject::OverwriteObject(
+    core::PropertyBuilder::createProperty("Overwrite Object")
+        ->withDescription("If false, the upload to GCS will succeed only if the object does not exist.")
+        ->withDefaultValue<bool>(true)
+        ->build());
+
+const core::Relationship PutGCSObject::Success("success", "Files that have been successfully written to Google Cloud Storage are transferred to this relationship");
+const core::Relationship PutGCSObject::Failure("failure", "Files that could not be written to Google Cloud Storage for some reason are transferred to this relationship");
+
+REGISTER_RESOURCE(PutGCSObject, Processor);
+
+}  // namespace org::apache::nifi::minifi::extensions::gcp
diff --git a/extensions/gcp/processors/ListGCSBucket.cpp b/extensions/gcp/processors/ListGCSBucket.cpp
index be6dc2204..e1bbcf5bc 100644
--- a/extensions/gcp/processors/ListGCSBucket.cpp
+++ b/extensions/gcp/processors/ListGCSBucket.cpp
@@ -26,28 +26,9 @@
 namespace gcs = ::google::cloud::storage;
 
 namespace org::apache::nifi::minifi::extensions::gcp {
-const core::Property ListGCSBucket::Bucket(
-    core::PropertyBuilder::createProperty("Bucket")
-        ->withDescription("Bucket of the object.")
-        ->isRequired(true)
-        ->supportsExpressionLanguage(true)
-        ->build());
-
-const core::Property ListGCSBucket::ListAllVersions(
-    core::PropertyBuilder::createProperty("List all versions")
-        ->withDescription("Set this option to `true` to get all the previous versions separately.")
-        ->withDefaultValue<bool>(false)
-        ->build());
-
-const core::Relationship ListGCSBucket::Success("success", "FlowFiles are routed to this relationship after a successful Google Cloud Storage operation.");
-
 void ListGCSBucket::initialize() {
-  setSupportedProperties({GCPCredentials,
-                          Bucket,
-                          NumberOfRetries,
-                          EndpointOverrideURL,
-                          ListAllVersions});
-  setSupportedRelationships({Success});
+  setSupportedProperties(properties());
+  setSupportedRelationships(relationships());
 }
 
 
@@ -75,7 +56,4 @@ void ListGCSBucket::onTrigger(const std::shared_ptr<core::ProcessContext>& conte
     }
   }
 }
-
-REGISTER_RESOURCE(ListGCSBucket, "Retrieves a listing of objects from an GCS bucket. "
-                                 "For each object that is listed, creates a FlowFile that represents the object so that it can be fetched in conjunction with FetchGCSObject.");
 }  // namespace org::apache::nifi::minifi::extensions::gcp
diff --git a/extensions/gcp/processors/ListGCSBucket.h b/extensions/gcp/processors/ListGCSBucket.h
index bdf8555f0..d2ca79a38 100644
--- a/extensions/gcp/processors/ListGCSBucket.h
+++ b/extensions/gcp/processors/ListGCSBucket.h
@@ -22,6 +22,7 @@
 
 #include "GCSProcessor.h"
 #include "core/logging/LoggerConfiguration.h"
+#include "utils/ArrayUtils.h"
 
 namespace org::apache::nifi::minifi::extensions::gcp {
 
@@ -32,23 +33,32 @@ class ListGCSBucket : public GCSProcessor {
   }
   ~ListGCSBucket() override = default;
 
+  EXTENSIONAPI static constexpr const char* Description = "Retrieves a listing of objects from an GCS bucket. "
+      "For each object that is listed, creates a FlowFile that represents the object so that it can be fetched in conjunction with FetchGCSObject.";
+
   EXTENSIONAPI static const core::Property Bucket;
   EXTENSIONAPI static const core::Property ListAllVersions;
+  static auto properties() {
+    return utils::array_cat(GCSProcessor::properties(), std::array{
+      Bucket,
+      ListAllVersions
+    });
+  }
 
   EXTENSIONAPI static const core::Relationship Success;
+  static auto relationships() { return std::array{Success}; }
+
+  EXTENSIONAPI static constexpr bool SupportsDynamicProperties = false;
+  EXTENSIONAPI static constexpr bool SupportsDynamicRelationships = false;
+  EXTENSIONAPI static constexpr core::annotation::Input InputRequirement = core::annotation::Input::INPUT_FORBIDDEN;
+  EXTENSIONAPI static constexpr bool IsSingleThreaded = true;
+
+  ADD_COMMON_VIRTUAL_FUNCTIONS_FOR_PROCESSORS
 
   void initialize() override;
   void onSchedule(const std::shared_ptr<core::ProcessContext> &context, const std::shared_ptr<core::ProcessSessionFactory> &sessionFactory) override;
   void onTrigger(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSession>& session) override;
 
-  core::annotation::Input getInputRequirement() const override {
-    return core::annotation::Input::INPUT_FORBIDDEN;
-  }
-
-  bool isSingleThreaded() const override {
-    return true;
-  }
-
  private:
   std::string bucket_;
 };
diff --git a/extensions/gcp/processors/PutGCSObject.cpp b/extensions/gcp/processors/PutGCSObject.cpp
index eb5f4991d..9578e0896 100644
--- a/extensions/gcp/processors/PutGCSObject.cpp
+++ b/extensions/gcp/processors/PutGCSObject.cpp
@@ -28,65 +28,6 @@
 namespace gcs = ::google::cloud::storage;
 
 namespace org::apache::nifi::minifi::extensions::gcp {
-const core::Property PutGCSObject::Bucket(
-    core::PropertyBuilder::createProperty("Bucket")
-        ->withDescription("Bucket of the object.")
-        ->withDefaultValue("${gcs.bucket}")
-        ->supportsExpressionLanguage(true)
-        ->build());
-
-const core::Property PutGCSObject::Key(
-    core::PropertyBuilder::createProperty("Key")
-        ->withDescription("Name of the object.")
-        ->withDefaultValue("${filename}")
-        ->supportsExpressionLanguage(true)
-        ->build());
-
-const core::Property PutGCSObject::ContentType(
-    core::PropertyBuilder::createProperty("Content Type")
-        ->withDescription("Content Type for the file, i.e. text/plain ")
-        ->isRequired(false)
-        ->withDefaultValue("${mime.type}")
-        ->supportsExpressionLanguage(true)
-        ->build());
-
-const core::Property PutGCSObject::MD5Hash(
-    core::PropertyBuilder::createProperty("MD5 Hash")
-        ->withDescription("MD5 Hash (encoded in Base64) of the file for server-side validation.")
-        ->isRequired(false)
-        ->supportsExpressionLanguage(true)
-        ->build());
-
-const core::Property PutGCSObject::Crc32cChecksum(
-    core::PropertyBuilder::createProperty("CRC32C Checksum")
-        ->withDescription("CRC32C Checksum (encoded in Base64, big-Endian order) of the file for server-side validation.")
-        ->isRequired(false)
-        ->supportsExpressionLanguage(true)
-        ->build());
-
-const core::Property PutGCSObject::EncryptionKey(
-    core::PropertyBuilder::createProperty("Server Side Encryption Key")
-        ->withDescription("An AES256 Encryption Key (encoded in base64) for server-side encryption of the object.")
-        ->isRequired(false)
-        ->supportsExpressionLanguage(true)
-        ->build());
-
-const core::Property PutGCSObject::ObjectACL(
-    core::PropertyBuilder::createProperty("Object ACL")
-        ->withDescription("Access Control to be attached to the object uploaded. Not providing this will revert to bucket defaults.")
-        ->isRequired(false)
-        ->withAllowableValues(PredefinedAcl::values())
-        ->build());
-
-const core::Property PutGCSObject::OverwriteObject(
-    core::PropertyBuilder::createProperty("Overwrite Object")
-        ->withDescription("If false, the upload to GCS will succeed only if the object does not exist.")
-        ->withDefaultValue<bool>(true)
-        ->build());
-
-const core::Relationship PutGCSObject::Success("success", "Files that have been successfully written to Google Cloud Storage are transferred to this relationship");
-const core::Relationship PutGCSObject::Failure("failure", "Files that could not be written to Google Cloud Storage for some reason are transferred to this relationship");
-
 namespace {
 class UploadToGCSCallback {
  public:
@@ -161,18 +102,8 @@ class UploadToGCSCallback {
 
 
 void PutGCSObject::initialize() {
-  setSupportedProperties({GCPCredentials,
-                          Bucket,
-                          Key,
-                          NumberOfRetries,
-                          ContentType,
-                          MD5Hash,
-                          Crc32cChecksum,
-                          EncryptionKey,
-                          ObjectACL,
-                          OverwriteObject,
-                          EndpointOverrideURL});
-  setSupportedRelationships({Success, Failure});
+  setSupportedProperties(properties());
+  setSupportedRelationships(relationships());
 }
 
 
@@ -244,6 +175,4 @@ void PutGCSObject::onTrigger(const std::shared_ptr<core::ProcessContext>& contex
     session->transfer(flow_file, Success);
   }
 }
-
-REGISTER_RESOURCE(PutGCSObject, "Puts flow files to a Google Cloud Storage Bucket.");
 }  // namespace org::apache::nifi::minifi::extensions::gcp
diff --git a/extensions/gcp/processors/PutGCSObject.h b/extensions/gcp/processors/PutGCSObject.h
index d9cd68019..42d47d560 100644
--- a/extensions/gcp/processors/PutGCSObject.h
+++ b/extensions/gcp/processors/PutGCSObject.h
@@ -22,6 +22,7 @@
 
 #include "GCSProcessor.h"
 #include "core/logging/LoggerConfiguration.h"
+#include "utils/ArrayUtils.h"
 #include "utils/Enum.h"
 #include "google/cloud/storage/well_known_headers.h"
 
@@ -43,6 +44,8 @@ class PutGCSObject : public GCSProcessor {
   }
   ~PutGCSObject() override = default;
 
+  EXTENSIONAPI static constexpr const char* Description = "Puts flow files to a Google Cloud Storage Bucket.";
+
   EXTENSIONAPI static const core::Property Bucket;
   EXTENSIONAPI static const core::Property Key;
   EXTENSIONAPI static const core::Property ContentType;
@@ -51,22 +54,35 @@ class PutGCSObject : public GCSProcessor {
   EXTENSIONAPI static const core::Property EncryptionKey;
   EXTENSIONAPI static const core::Property ObjectACL;
   EXTENSIONAPI static const core::Property OverwriteObject;
+  static auto properties() {
+    return utils::array_cat(GCSProcessor::properties(), std::array{
+      Bucket,
+      Key,
+      NumberOfRetries,
+      ContentType,
+      MD5Hash,
+      Crc32cChecksum,
+      EncryptionKey,
+      ObjectACL,
+      OverwriteObject
+    });
+  }
 
   EXTENSIONAPI static const core::Relationship Success;
   EXTENSIONAPI static const core::Relationship Failure;
+  static auto relationships() { return std::array{Success, Failure}; }
+
+  EXTENSIONAPI static constexpr bool SupportsDynamicProperties = false;
+  EXTENSIONAPI static constexpr bool SupportsDynamicRelationships = false;
+  EXTENSIONAPI static constexpr core::annotation::Input InputRequirement = core::annotation::Input::INPUT_REQUIRED;
+  EXTENSIONAPI static constexpr bool IsSingleThreaded = true;
+
+  ADD_COMMON_VIRTUAL_FUNCTIONS_FOR_PROCESSORS
 
   void initialize() override;
   void onSchedule(const std::shared_ptr<core::ProcessContext> &context, const std::shared_ptr<core::ProcessSessionFactory> &sessionFactory) override;
   void onTrigger(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSession>& session) override;
 
-  core::annotation::Input getInputRequirement() const override {
-    return core::annotation::Input::INPUT_REQUIRED;
-  }
-
-  bool isSingleThreaded() const override {
-    return true;
-  }
-
  private:
   google::cloud::storage::EncryptionKey encryption_key_;
 };
diff --git a/extensions/gcp/tests/DeleteGCSObjectTests.cpp b/extensions/gcp/tests/DeleteGCSObjectTests.cpp
index 2b0e446fb..777c488cc 100644
--- a/extensions/gcp/tests/DeleteGCSObjectTests.cpp
+++ b/extensions/gcp/tests/DeleteGCSObjectTests.cpp
@@ -36,12 +36,14 @@ namespace {
 class DeleteGCSObjectMocked : public DeleteGCSObject {
   using org::apache::nifi::minifi::extensions::gcp::DeleteGCSObject::DeleteGCSObject;
  public:
+  static constexpr const char* Description = "DeleteGCSObjectMocked";
+
   gcs::Client getClient() const override {
     return gcs::testing::ClientFromMock(mock_client_, *retry_policy_);
   }
   std::shared_ptr<gcs::testing::MockClient> mock_client_ = std::make_shared<gcs::testing::MockClient>();
 };
-REGISTER_RESOURCE(DeleteGCSObjectMocked, "DeleteGCSObjectMocked");
+REGISTER_RESOURCE(DeleteGCSObjectMocked, Processor);
 }  // namespace
 
 class DeleteGCSObjectTests : public ::testing::Test {
diff --git a/extensions/gcp/tests/FetchGCSObjectTests.cpp b/extensions/gcp/tests/FetchGCSObjectTests.cpp
index 696d3b4dc..5b46033b7 100644
--- a/extensions/gcp/tests/FetchGCSObjectTests.cpp
+++ b/extensions/gcp/tests/FetchGCSObjectTests.cpp
@@ -35,12 +35,14 @@ namespace {
 class FetchGCSObjectMocked : public FetchGCSObject {
   using org::apache::nifi::minifi::extensions::gcp::FetchGCSObject::FetchGCSObject;
  public:
+  static constexpr const char* Description = "FetchGCSObjectMocked";
+
   gcs::Client getClient() const override {
     return gcs::testing::ClientFromMock(mock_client_, *retry_policy_);
   }
   std::shared_ptr<gcs::testing::MockClient> mock_client_ = std::make_shared<gcs::testing::MockClient>();
 };
-REGISTER_RESOURCE(FetchGCSObjectMocked, "FetchGCSObjectMocked");
+REGISTER_RESOURCE(FetchGCSObjectMocked, Processor);
 }  // namespace
 
 class FetchGCSObjectTests : public ::testing::Test {
diff --git a/extensions/gcp/tests/GCPCredentialsControllerServiceTests.cpp b/extensions/gcp/tests/GCPCredentialsControllerServiceTests.cpp
index f6e3b7185..3f5051cac 100644
--- a/extensions/gcp/tests/GCPCredentialsControllerServiceTests.cpp
+++ b/extensions/gcp/tests/GCPCredentialsControllerServiceTests.cpp
@@ -66,10 +66,20 @@ std::optional<std::filesystem::path> create_mock_json_file(const std::filesystem
 }
 
 class DummyProcessor : public org::apache::nifi::minifi::core::Processor {
+ public:
   using minifi::core::Processor::Processor;
+
+  static constexpr const char* Description = "A processor that does nothing.";
+  static auto properties() { return std::array<core::Property, 0>{}; }
+  static auto relationships() { return std::array<core::Relationship, 0>{}; }
+  static constexpr bool SupportsDynamicProperties = false;
+  static constexpr bool SupportsDynamicRelationships = false;
+  static constexpr core::annotation::Input InputRequirement = core::annotation::Input::INPUT_ALLOWED;
+  static constexpr bool IsSingleThreaded = false;
+  ADD_COMMON_VIRTUAL_FUNCTIONS_FOR_PROCESSORS
 };
 
-REGISTER_RESOURCE(DummyProcessor, "A processor that does nothing.");
+REGISTER_RESOURCE(DummyProcessor, Processor);
 }  // namespace
 
 class GCPCredentialsTests : public ::testing::Test {
diff --git a/extensions/gcp/tests/ListGCSBucketTests.cpp b/extensions/gcp/tests/ListGCSBucketTests.cpp
index a5851945a..a1e7e92d7 100644
--- a/extensions/gcp/tests/ListGCSBucketTests.cpp
+++ b/extensions/gcp/tests/ListGCSBucketTests.cpp
@@ -36,12 +36,14 @@ namespace {
 class ListGCSBucketMocked : public ListGCSBucket {
   using org::apache::nifi::minifi::extensions::gcp::ListGCSBucket::ListGCSBucket;
  public:
+  static constexpr const char* Description = "ListGCSBucketMocked";
+
   gcs::Client getClient() const override {
     return gcs::testing::ClientFromMock(mock_client_, *retry_policy_);
   }
   std::shared_ptr<gcs::testing::MockClient> mock_client_ = std::make_shared<gcs::testing::MockClient>();
 };
-REGISTER_RESOURCE(ListGCSBucketMocked, "ListGCSBucketMocked");
+REGISTER_RESOURCE(ListGCSBucketMocked, Processor);
 
 auto CreateObject(int index, int generation = 1) {
   std::string id = "object-" + std::to_string(index);
diff --git a/extensions/gcp/tests/PutGCSObjectTests.cpp b/extensions/gcp/tests/PutGCSObjectTests.cpp
index a5c891906..2d0f06008 100644
--- a/extensions/gcp/tests/PutGCSObjectTests.cpp
+++ b/extensions/gcp/tests/PutGCSObjectTests.cpp
@@ -40,12 +40,14 @@ namespace {
 class PutGCSObjectMocked : public PutGCSObject {
   using org::apache::nifi::minifi::extensions::gcp::PutGCSObject::PutGCSObject;
  public:
+  static constexpr const char* Description = "PutGCSObjectMocked";
+
   gcs::Client getClient() const override {
     return gcs::testing::ClientFromMock(mock_client_, *retry_policy_);
   }
   std::shared_ptr<gcs::testing::MockClient> mock_client_ = std::make_shared<gcs::testing::MockClient>();
 };
-REGISTER_RESOURCE(PutGCSObjectMocked, "PutGCSObjectMocked");
+REGISTER_RESOURCE(PutGCSObjectMocked, Processor);
 }  // namespace
 
 class PutGCSObjectTests : public ::testing::Test {
diff --git a/extensions/gps/GetGPS.cpp b/extensions/gps/GetGPS.cpp
index a8af0bdaa..63f0a4b43 100644
--- a/extensions/gps/GetGPS.cpp
+++ b/extensions/gps/GetGPS.cpp
@@ -33,6 +33,7 @@
 
 #include "core/ProcessContext.h"
 #include "core/ProcessSession.h"
+#include "core/PropertyBuilder.h"
 #include "core/PropertyValidation.h"
 #include "core/Resource.h"
 
@@ -42,24 +43,17 @@ namespace nifi {
 namespace minifi {
 namespace processors {
 
-const std::string GetGPS::ProcessorName("GetGPS");
-core::Relationship GetGPS::Success("success", "All files are routed to success");
-core::Property GetGPS::GPSDHost(core::PropertyBuilder::createProperty("GPSD Host")->withDescription("The host running the GPSD daemon")->withDefaultValue<std::string>("localhost")->build());
-core::Property GetGPS::GPSDPort(
+const core::Relationship GetGPS::Success("success", "All files are routed to success");
+
+const core::Property GetGPS::GPSDHost(core::PropertyBuilder::createProperty("GPSD Host")->withDescription("The host running the GPSD daemon")->withDefaultValue<std::string>("localhost")->build());
+const core::Property GetGPS::GPSDPort(
     core::PropertyBuilder::createProperty("GPSD Port")->withDescription("The GPSD daemon port")->withDefaultValue<int64_t>(2947, core::StandardValidators::get().PORT_VALIDATOR)->build());
-core::Property GetGPS::GPSDWaitTime(
+const core::Property GetGPS::GPSDWaitTime(
     core::PropertyBuilder::createProperty("GPSD Wait Time")->withDescription("Timeout value for waiting for data from the GPSD instance")->withDefaultValue<uint64_t>(50000000)->build());
+
 void GetGPS::initialize() {
-  //! Set the supported properties
-  std::set<core::Property> properties;
-  properties.insert(GPSDHost);
-  properties.insert(GPSDPort);
-  properties.insert(GPSDWaitTime);
-  setSupportedProperties(properties);
-  //! Set the supported relationships
-  std::set<core::Relationship> relationships;
-  relationships.insert(Success);
-  setSupportedRelationships(relationships);
+  setSupportedProperties(properties());
+  setSupportedRelationships(relationships());
 }
 
 void GetGPS::onSchedule(const std::shared_ptr<core::ProcessContext> &context, const std::shared_ptr<core::ProcessSessionFactory>& /*sessionFactory*/) {
@@ -150,7 +144,7 @@ void GetGPS::onTrigger(const std::shared_ptr<core::ProcessContext>& /*context*/,
   }
 }
 
-REGISTER_RESOURCE(GetGPS, "Obtains GPS coordinates from the GPSDHost and port.");
+REGISTER_RESOURCE(GetGPS, Processor);
 
 } /* namespace processors */
 } /* namespace minifi */
diff --git a/extensions/gps/GetGPS.h b/extensions/gps/GetGPS.h
index 73f61d7bd..1ade4d149 100644
--- a/extensions/gps/GetGPS.h
+++ b/extensions/gps/GetGPS.h
@@ -40,33 +40,38 @@ class GetGPS : public core::Processor {
     gpsdWaitTime_ = 50000000;
   }
   ~GetGPS() override = default;
-  static const std::string ProcessorName;
-  // Supported Properties
-  static core::Property GPSDHost;
-  static core::Property GPSDPort;
-  static core::Property GPSDWaitTime;
 
-  // Supported Relationships
-  static core::Relationship Success;
+  EXTENSIONAPI static constexpr const char* Description = "Obtains GPS coordinates from the GPSDHost and port.";
+
+  EXTENSIONAPI static const core::Property GPSDHost;
+  EXTENSIONAPI static const core::Property GPSDPort;
+  EXTENSIONAPI static const core::Property GPSDWaitTime;
+  static auto properties() {
+    return std::array{
+      GPSDHost,
+      GPSDPort,
+      GPSDWaitTime
+    };
+  }
+
+  EXTENSIONAPI static const core::Relationship Success;
+  static auto relationships() { return std::array{Success}; }
+
+  EXTENSIONAPI static constexpr bool SupportsDynamicProperties = false;
+  EXTENSIONAPI static constexpr bool SupportsDynamicRelationships = false;
+  EXTENSIONAPI static constexpr core::annotation::Input InputRequirement = core::annotation::Input::INPUT_ALLOWED;
+  EXTENSIONAPI static constexpr bool IsSingleThreaded = false;
+
+  ADD_COMMON_VIRTUAL_FUNCTIONS_FOR_PROCESSORS
 
- public:
-  /**
-   * Function that's executed when the processor is scheduled.
-   * @param context process context.
-   * @param sessionFactory process session factory that is used when creating
-   * ProcessSession objects.
-   */
   void onSchedule(const std::shared_ptr<core::ProcessContext> &context, const std::shared_ptr<core::ProcessSessionFactory> &sessionFactory) override;
-  //! OnTrigger method, implemented by NiFi GetGPS
   void onTrigger(const std::shared_ptr<core::ProcessContext> &context, const std::shared_ptr<core::ProcessSession> &session) override;
-  //! Initialize, over write by NiFi GetGPS
   void initialize() override;
 
  private:
   std::string gpsdHost_;
   std::string gpsdPort_;
   int64_t gpsdWaitTime_;
-  // Logger
   std::shared_ptr<core::logging::Logger> logger_ = core::logging::LoggerFactory<GetGPS>::getLogger();
 };
 
diff --git a/extensions/http-curl/client/HTTPClient.cpp b/extensions/http-curl/client/HTTPClient.cpp
index fe4712318..dd28e47be 100644
--- a/extensions/http-curl/client/HTTPClient.cpp
+++ b/extensions/http-curl/client/HTTPClient.cpp
@@ -1,5 +1,4 @@
 /**
- *
  * 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.
@@ -470,6 +469,6 @@ std::string HTTPClient::replaceInvalidCharactersInHttpHeaderFieldName(std::strin
   return result;
 }
 
-REGISTER_INTERNAL_RESOURCE(HTTPClient);
+REGISTER_RESOURCE(HTTPClient, InternalResource);
 
 }  // namespace org::apache::nifi::minifi::utils
diff --git a/extensions/http-curl/client/HTTPClient.h b/extensions/http-curl/client/HTTPClient.h
index 16491be46..29f60b0e7 100644
--- a/extensions/http-curl/client/HTTPClient.h
+++ b/extensions/http-curl/client/HTTPClient.h
@@ -1,5 +1,4 @@
 /**
- *
  * 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.
@@ -15,17 +14,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+#pragma once
 
-// Visual Studio 2017 warns when overriding a deprecated function, even if
-// the override is also deprecated.  Note that we need to put this #pragma
-// here, because it doesn't work inside the #ifndef
 #ifdef WIN32
 #pragma warning(push)
 #pragma warning(disable: 4996)
 #endif
 
-#pragma once
-
 #include "utils/HTTPClient.h"
 #ifdef WIN32
 #pragma comment(lib, "wldap32.lib" )
@@ -77,7 +72,11 @@ class HTTPClient : public BaseHTTPClient, public core::Connectable {
 
   explicit HTTPClient(const std::string &url, const std::shared_ptr<minifi::controllers::SSLContextService> ssl_context_service = nullptr);
 
-  ~HTTPClient();
+  ~HTTPClient() override;
+
+  static auto properties() { return std::array<core::Property, 0>{}; }
+  EXTENSIONAPI static constexpr bool SupportsDynamicProperties = false;
+  EXTENSIONAPI static constexpr bool SupportsDynamicRelationships = false;
 
   static int debug_callback(CURL *handle, curl_infotype type, char *data, size_t size, void *userptr);
 
diff --git a/extensions/http-curl/processors/InvokeHTTP.cpp b/extensions/http-curl/processors/InvokeHTTP.cpp
index 2177d2c89..65ef36afa 100644
--- a/extensions/http-curl/processors/InvokeHTTP.cpp
+++ b/extensions/http-curl/processors/InvokeHTTP.cpp
@@ -21,7 +21,6 @@
 #include <cinttypes>
 #include <cstdint>
 #include <memory>
-#include <set>
 #include <string>
 #include <utility>
 #include <vector>
@@ -29,6 +28,7 @@
 #include "utils/ByteArrayCallback.h"
 #include "core/FlowFile.h"
 #include "core/ProcessContext.h"
+#include "core/PropertyBuilder.h"
 #include "core/Relationship.h"
 #include "core/Resource.h"
 #include "io/BufferStream.h"
@@ -45,74 +45,74 @@ namespace org::apache::nifi::minifi::processors {
 
 std::string InvokeHTTP::DefaultContentType = "application/octet-stream";
 
-core::Property InvokeHTTP::Method("HTTP Method", "HTTP request method (GET, POST, PUT, PATCH, DELETE, HEAD, OPTIONS). "
+const core::Property InvokeHTTP::Method("HTTP Method", "HTTP request method (GET, POST, PUT, PATCH, DELETE, HEAD, OPTIONS). "
                                   "Arbitrary methods are also supported. Methods other than POST, PUT and PATCH will be sent without a message body.",
                                   "GET");
-core::Property InvokeHTTP::URL(
+const core::Property InvokeHTTP::URL(
     core::PropertyBuilder::createProperty("Remote URL")->withDescription("Remote URL which will be connected to, including scheme, host, port, path.")->isRequired(false)->supportsExpressionLanguage(
         true)->build());
 
-core::Property InvokeHTTP::ConnectTimeout(
+const core::Property InvokeHTTP::ConnectTimeout(
       core::PropertyBuilder::createProperty("Connection Timeout")->withDescription("Max wait time for connection to remote service")->isRequired(false)
          ->withDefaultValue<core::TimePeriodValue>("5 s")->build());
 
-core::Property InvokeHTTP::ReadTimeout(
+const core::Property InvokeHTTP::ReadTimeout(
       core::PropertyBuilder::createProperty("Read Timeout")->withDescription("Max wait time for response from remote service")->isRequired(false)
          ->withDefaultValue<core::TimePeriodValue>("15 s")->build());
 
-core::Property InvokeHTTP::DateHeader(
+const core::Property InvokeHTTP::DateHeader(
     core::PropertyBuilder::createProperty("Include Date Header")->withDescription("Include an RFC-2616 Date header in the request.")->isRequired(false)->withDefaultValue<bool>(true)->build());
 
-core::Property InvokeHTTP::FollowRedirects(
+const core::Property InvokeHTTP::FollowRedirects(
   core::PropertyBuilder::createProperty("Follow Redirects")
   ->withDescription("Follow HTTP redirects issued by remote server.")
   ->withDefaultValue<bool>(true)
   ->build());
-core::Property InvokeHTTP::AttributesToSend("Attributes to Send", "Regular expression that defines which attributes to send as HTTP"
+const core::Property InvokeHTTP::AttributesToSend("Attributes to Send", "Regular expression that defines which attributes to send as HTTP"
                                             " headers in the request. If not defined, no attributes are sent as headers.",
                                             "");
-core::Property InvokeHTTP::SSLContext(
+const core::Property InvokeHTTP::SSLContext(
     core::PropertyBuilder::createProperty("SSL Context Service")->withDescription("The SSL Context Service used to provide client certificate "
                                                                                   "information for TLS/SSL (https) connections.")->isRequired(false)->withExclusiveProperty("Remote URL", "^http:.*$")
         ->asType<minifi::controllers::SSLContextService>()->build());
-core::Property InvokeHTTP::ProxyHost("Proxy Host", "The fully qualified hostname or IP address of the proxy server", "");
-core::Property InvokeHTTP::ProxyPort(
+const core::Property InvokeHTTP::ProxyHost("Proxy Host", "The fully qualified hostname or IP address of the proxy server", "");
+const core::Property InvokeHTTP::ProxyPort(
     core::PropertyBuilder::createProperty("Proxy Port")->withDescription("The port of the proxy server")
         ->isRequired(false)->build());
-core::Property InvokeHTTP::ProxyUsername(
+const core::Property InvokeHTTP::ProxyUsername(
     core::PropertyBuilder::createProperty("invokehttp-proxy-username", "Proxy Username")->withDescription("Username to set when authenticating against proxy")->isRequired(false)->build());
-core::Property InvokeHTTP::ProxyPassword(
+const core::Property InvokeHTTP::ProxyPassword(
     core::PropertyBuilder::createProperty("invokehttp-proxy-password", "Proxy Password")->withDescription("Password to set when authenticating against proxy")->isRequired(false)->build());
-core::Property InvokeHTTP::ContentType("Content-type", "The Content-Type to specify for when content is being transmitted through a PUT, "
+const core::Property InvokeHTTP::ContentType("Content-type", "The Content-Type to specify for when content is being transmitted through a PUT, "
                                        "POST or PATCH. In the case of an empty value after evaluating an expression language expression, "
                                        "Content-Type defaults to",
                                        "application/octet-stream");
-core::Property InvokeHTTP::SendBody(
+const core::Property InvokeHTTP::SendBody(
     core::PropertyBuilder::createProperty("send-message-body", "Send Body")
       ->withDescription("DEPRECATED. Only kept for backwards compatibility, no functionality is included.")
       ->withDefaultValue<bool>(true)
       ->build());
-core::Property InvokeHTTP::SendMessageBody(
+const core::Property InvokeHTTP::SendMessageBody(
     core::PropertyBuilder::createProperty("Send Message Body")
       ->withDescription("If true, sends the HTTP message body on POST/PUT/PATCH requests (default). "
                         "If false, suppresses the message body and content-type header for these requests.")
       ->withDefaultValue<bool>(true)
       ->build());
-core::Property InvokeHTTP::UseChunkedEncoding("Use Chunked Encoding", "When POST'ing, PUT'ing or PATCH'ing content set this property to true in order to not pass the 'Content-length' header"
+const core::Property InvokeHTTP::UseChunkedEncoding("Use Chunked Encoding", "When POST'ing, PUT'ing or PATCH'ing content set this property to true in order to not pass the 'Content-length' header"
                                               " and instead send 'Transfer-Encoding' with a value of 'chunked'. This will enable the data transfer mechanism which was introduced in HTTP 1.1 "
                                               "to pass data of unknown lengths in chunks.",
                                               "false");
-core::Property InvokeHTTP::PropPutOutputAttributes("Put Response Body in Attribute", "If set, the response body received back will be put into an attribute of the original "
+const core::Property InvokeHTTP::PropPutOutputAttributes("Put Response Body in Attribute", "If set, the response body received back will be put into an attribute of the original "
                                                    "FlowFile instead of a separate FlowFile. The attribute key to put to is determined by evaluating value of this property. ",
                                                    "");
-core::Property InvokeHTTP::AlwaysOutputResponse("Always Output Response", "Will force a response FlowFile to be generated and routed to the 'Response' relationship "
+const core::Property InvokeHTTP::AlwaysOutputResponse("Always Output Response", "Will force a response FlowFile to be generated and routed to the 'Response' relationship "
                                                 "regardless of what the server status code received is ",
                                                 "false");
-core::Property InvokeHTTP::PenalizeOnNoRetry("Penalize on \"No Retry\"", "Enabling this property will penalize FlowFiles that are routed to the \"No Retry\" relationship.", "false");
+const core::Property InvokeHTTP::PenalizeOnNoRetry("Penalize on \"No Retry\"", "Enabling this property will penalize FlowFiles that are routed to the \"No Retry\" relationship.", "false");
 
-core::Property InvokeHTTP::DisablePeerVerification("Disable Peer Verification", "Disables peer verification for the SSL session", "false");
+const core::Property InvokeHTTP::DisablePeerVerification("Disable Peer Verification", "Disables peer verification for the SSL session", "false");
 
-core::Property InvokeHTTP::InvalidHTTPHeaderFieldHandlingStrategy(
+const core::Property InvokeHTTP::InvalidHTTPHeaderFieldHandlingStrategy(
     core::PropertyBuilder::createProperty("Invalid HTTP Header Field Handling Strategy")
       ->withDescription("Indicates what should happen when an attribute's name is not a valid HTTP header field name. "
         "Options: transform - invalid characters are replaced, fail - flow file is transferred to failure, drop - drops invalid attributes from HTTP message")
@@ -121,57 +121,27 @@ core::Property InvokeHTTP::InvalidHTTPHeaderFieldHandlingStrategy(
       ->withAllowableValues<std::string>(InvalidHTTPHeaderFieldHandlingOption::values())
       ->build());
 
-const char* InvokeHTTP::STATUS_CODE = "invokehttp.status.code";
-const char* InvokeHTTP::STATUS_MESSAGE = "invokehttp.status.message";
-const char* InvokeHTTP::RESPONSE_BODY = "invokehttp.response.body";
-const char* InvokeHTTP::REQUEST_URL = "invokehttp.request.url";
-const char* InvokeHTTP::TRANSACTION_ID = "invokehttp.tx.id";
-const char* InvokeHTTP::REMOTE_DN = "invokehttp.remote.dn";
-const char* InvokeHTTP::EXCEPTION_CLASS = "invokehttp.java.exception.class";
-const char* InvokeHTTP::EXCEPTION_MESSAGE = "invokehttp.java.exception.message";
 
-core::Relationship InvokeHTTP::Success("success", "The original FlowFile will be routed upon success (2xx status codes). "
+const core::Relationship InvokeHTTP::Success("success", "The original FlowFile will be routed upon success (2xx status codes). "
                                        "It will have new attributes detailing the success of the request.");
 
-core::Relationship InvokeHTTP::RelResponse("response", "A Response FlowFile will be routed upon success (2xx status codes). "
+const core::Relationship InvokeHTTP::RelResponse("response", "A Response FlowFile will be routed upon success (2xx status codes). "
                                            "If the 'Always Output Response' property is true then the response will be sent "
                                            "to this relationship regardless of the status code received.");
 
-core::Relationship InvokeHTTP::RelRetry("retry", "The original FlowFile will be routed on any status code that can be retried "
+const core::Relationship InvokeHTTP::RelRetry("retry", "The original FlowFile will be routed on any status code that can be retried "
                                         "(5xx status codes). It will have new attributes detailing the request.");
 
-core::Relationship InvokeHTTP::RelNoRetry("no retry", "The original FlowFile will be routed on any status code that should NOT "
+const core::Relationship InvokeHTTP::RelNoRetry("no retry", "The original FlowFile will be routed on any status code that should NOT "
                                           "be retried (1xx, 3xx, 4xx status codes). It will have new attributes detailing the request.");
 
-core::Relationship InvokeHTTP::RelFailure("failure", "The original FlowFile will be routed on any type of connection failure, "
+const core::Relationship InvokeHTTP::RelFailure("failure", "The original FlowFile will be routed on any type of connection failure, "
                                           "timeout or general exception. It will have new attributes detailing the request.");
 
 void InvokeHTTP::initialize() {
   logger_->log_trace("Initializing InvokeHTTP");
-  setSupportedProperties({
-    Method,
-    URL,
-    ConnectTimeout,
-    ReadTimeout,
-    DateHeader,
-    AttributesToSend,
-    SSLContext,
-    ProxyHost,
-    ProxyPort,
-    ProxyUsername,
-    ProxyPassword,
-    UseChunkedEncoding,
-    ContentType,
-    SendBody,
-    SendMessageBody,
-    DisablePeerVerification,
-    AlwaysOutputResponse,
-    FollowRedirects,
-    PropPutOutputAttributes,
-    PenalizeOnNoRetry,
-    InvalidHTTPHeaderFieldHandlingStrategy
-  });
-  setSupportedRelationships({Success, RelResponse, RelFailure, RelRetry, RelNoRetry});
+  setSupportedProperties(properties());
+  setSupportedRelationships(relationships());
 }
 
 void InvokeHTTP::onSchedule(const std::shared_ptr<core::ProcessContext> &context, const std::shared_ptr<core::ProcessSessionFactory>& /*sessionFactory*/) {
@@ -483,8 +453,6 @@ void InvokeHTTP::route(const std::shared_ptr<core::FlowFile> &request, const std
   }
 }
 
-REGISTER_RESOURCE(InvokeHTTP, "An HTTP client processor which can interact with a configurable HTTP Endpoint. "
-    "The destination URL and HTTP Method are configurable. FlowFile attributes are converted to HTTP headers and the "
-    "FlowFile contents are included as the body of the request (if the HTTP Method is PUT, POST or PATCH).");
+REGISTER_RESOURCE(InvokeHTTP, Processor);
 
 }  // namespace org::apache::nifi::minifi::processors
diff --git a/extensions/http-curl/processors/InvokeHTTP.h b/extensions/http-curl/processors/InvokeHTTP.h
index 6320536c8..4e11150c3 100644
--- a/extensions/http-curl/processors/InvokeHTTP.h
+++ b/extensions/http-curl/processors/InvokeHTTP.h
@@ -49,44 +49,90 @@ class InvokeHTTP : public core::Processor {
       : Processor(name, uuid) {
     setTriggerWhenEmpty(true);
   }
+
+  EXTENSIONAPI static constexpr const char* Description = "An HTTP client processor which can interact with a configurable HTTP Endpoint. "
+      "The destination URL and HTTP Method are configurable. FlowFile attributes are converted to HTTP headers and the "
+      "FlowFile contents are included as the body of the request (if the HTTP Method is PUT, POST or PATCH).";
+
+  EXTENSIONAPI static const core::Property Method;
+  EXTENSIONAPI static const core::Property URL;
+  EXTENSIONAPI static const core::Property ConnectTimeout;
+  EXTENSIONAPI static const core::Property ReadTimeout;
+  EXTENSIONAPI static const core::Property DateHeader;
+  EXTENSIONAPI static const core::Property FollowRedirects;
+  EXTENSIONAPI static const core::Property AttributesToSend;
+  EXTENSIONAPI static const core::Property SSLContext;
+  EXTENSIONAPI static const core::Property ProxyHost;
+  EXTENSIONAPI static const core::Property ProxyPort;
+  EXTENSIONAPI static const core::Property ProxyUsername;
+  EXTENSIONAPI static const core::Property ProxyPassword;
+  EXTENSIONAPI static const core::Property ContentType;
+  EXTENSIONAPI static const core::Property SendBody;
+  EXTENSIONAPI static const core::Property SendMessageBody;
+  EXTENSIONAPI static const core::Property UseChunkedEncoding;
+  EXTENSIONAPI static const core::Property DisablePeerVerification;
+  EXTENSIONAPI static const core::Property PropPutOutputAttributes;
+  EXTENSIONAPI static const core::Property AlwaysOutputResponse;
+  EXTENSIONAPI static const core::Property PenalizeOnNoRetry;
+  EXTENSIONAPI static const core::Property InvalidHTTPHeaderFieldHandlingStrategy;
+  static auto properties() {
+    return std::array{
+      Method,
+      URL,
+      ConnectTimeout,
+      ReadTimeout,
+      DateHeader,
+      FollowRedirects,
+      AttributesToSend,
+      SSLContext,
+      ProxyHost,
+      ProxyPort,
+      ProxyUsername,
+      ProxyPassword,
+      ContentType,
+      SendBody,
+      SendMessageBody,
+      UseChunkedEncoding,
+      DisablePeerVerification,
+      PropPutOutputAttributes,
+      AlwaysOutputResponse,
+      PenalizeOnNoRetry,
+      InvalidHTTPHeaderFieldHandlingStrategy
+    };
+  }
+
+  EXTENSIONAPI static const core::Relationship Success;
+  EXTENSIONAPI static const core::Relationship RelResponse;
+  EXTENSIONAPI static const core::Relationship RelRetry;
+  EXTENSIONAPI static const core::Relationship RelNoRetry;
+  EXTENSIONAPI static const core::Relationship RelFailure;
+  static auto relationships() {
+    return std::array{
+      Success,
+      RelResponse,
+      RelRetry,
+      RelNoRetry,
+      RelFailure
+    };
+  }
+
+  EXTENSIONAPI static constexpr bool SupportsDynamicProperties = false;
+  EXTENSIONAPI static constexpr bool SupportsDynamicRelationships = false;
+  EXTENSIONAPI static constexpr core::annotation::Input InputRequirement = core::annotation::Input::INPUT_ALLOWED;
+  EXTENSIONAPI static constexpr bool IsSingleThreaded = false;
+
+  ADD_COMMON_VIRTUAL_FUNCTIONS_FOR_PROCESSORS
+
   EXTENSIONAPI static std::string DefaultContentType;
 
-  EXTENSIONAPI static core::Property Method;
-  EXTENSIONAPI static core::Property URL;
-  EXTENSIONAPI static core::Property ConnectTimeout;
-  EXTENSIONAPI static core::Property ReadTimeout;
-  EXTENSIONAPI static core::Property DateHeader;
-  EXTENSIONAPI static core::Property FollowRedirects;
-  EXTENSIONAPI static core::Property AttributesToSend;
-  EXTENSIONAPI static core::Property SSLContext;
-  EXTENSIONAPI static core::Property ProxyHost;
-  EXTENSIONAPI static core::Property ProxyPort;
-  EXTENSIONAPI static core::Property ProxyUsername;
-  EXTENSIONAPI static core::Property ProxyPassword;
-  EXTENSIONAPI static core::Property ContentType;
-  EXTENSIONAPI static core::Property SendBody;
-  EXTENSIONAPI static core::Property SendMessageBody;
-  EXTENSIONAPI static core::Property UseChunkedEncoding;
-  EXTENSIONAPI static core::Property DisablePeerVerification;
-  EXTENSIONAPI static core::Property PropPutOutputAttributes;
-  EXTENSIONAPI static core::Property AlwaysOutputResponse;
-  EXTENSIONAPI static core::Property PenalizeOnNoRetry;
-  EXTENSIONAPI static core::Property InvalidHTTPHeaderFieldHandlingStrategy;
-
-  EXTENSIONAPI static const char* STATUS_CODE;
-  EXTENSIONAPI static const char* STATUS_MESSAGE;
-  EXTENSIONAPI static const char* RESPONSE_BODY;
-  EXTENSIONAPI static const char* REQUEST_URL;
-  EXTENSIONAPI static const char* TRANSACTION_ID;
-  EXTENSIONAPI static const char* REMOTE_DN;
-  EXTENSIONAPI static const char* EXCEPTION_CLASS;
-  EXTENSIONAPI static const char* EXCEPTION_MESSAGE;
-
-  EXTENSIONAPI static core::Relationship Success;
-  EXTENSIONAPI static core::Relationship RelResponse;
-  EXTENSIONAPI static core::Relationship RelRetry;
-  EXTENSIONAPI static core::Relationship RelNoRetry;
-  EXTENSIONAPI static core::Relationship RelFailure;
+  EXTENSIONAPI static constexpr const char* STATUS_CODE = "invokehttp.status.code";
+  EXTENSIONAPI static constexpr const char* STATUS_MESSAGE = "invokehttp.status.message";
+  EXTENSIONAPI static constexpr const char* RESPONSE_BODY = "invokehttp.response.body";
+  EXTENSIONAPI static constexpr const char* REQUEST_URL = "invokehttp.request.url";
+  EXTENSIONAPI static constexpr const char* TRANSACTION_ID = "invokehttp.tx.id";
+  EXTENSIONAPI static constexpr const char* REMOTE_DN = "invokehttp.remote.dn";
+  EXTENSIONAPI static constexpr const char* EXCEPTION_CLASS = "invokehttp.java.exception.class";
+  EXTENSIONAPI static constexpr const char* EXCEPTION_MESSAGE = "invokehttp.java.exception.message";
 
   void onTrigger(const std::shared_ptr<core::ProcessContext> &context, const std::shared_ptr<core::ProcessSession> &session) override;
   void initialize() override;
diff --git a/extensions/http-curl/protocols/AgentPrinter.cpp b/extensions/http-curl/protocols/AgentPrinter.cpp
index 7e1571593..f882f40c2 100644
--- a/extensions/http-curl/protocols/AgentPrinter.cpp
+++ b/extensions/http-curl/protocols/AgentPrinter.cpp
@@ -1,5 +1,4 @@
 /**
- *
  * 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.
@@ -25,18 +24,14 @@
 #include "rapidjson/prettywriter.h"
 #include "core/Resource.h"
 
-namespace org {
-namespace apache {
-namespace nifi {
-namespace minifi {
-namespace c2 {
+namespace org::apache::nifi::minifi::c2 {
 
 AgentPrinter::AgentPrinter(const std::string& name, const utils::Identifier& uuid)
-    : HeartbeatReporter(name, uuid) {
+: HeartbeatReporter(name, uuid) {
 }
 
 void AgentPrinter::initialize(core::controller::ControllerServiceProvider* controller, state::StateMonitor* updateSink,
-                              const std::shared_ptr<Configure> &configure) {
+    const std::shared_ptr<Configure> &configure) {
   HeartbeatReporter::initialize(controller, updateSink, configure);
 }
 int16_t AgentPrinter::heartbeat(const C2Payload &payload) {
@@ -60,10 +55,6 @@ rapidjson::Value AgentPrinter::serializeJsonPayload(const C2Payload &payload, ra
   return result;
 }
 
-REGISTER_RESOURCE(AgentPrinter, "Encapsulates printing agent information.");
+REGISTER_RESOURCE(AgentPrinter, DescriptionOnly);
 
-} /* namespace c2 */
-} /* namespace minifi */
-} /* namespace nifi */
-} /* namespace apache */
-} /* namespace org */
+}  // namespace org::apache::nifi::minifi::c2
diff --git a/extensions/http-curl/protocols/AgentPrinter.h b/extensions/http-curl/protocols/AgentPrinter.h
index fa3680504..e384b85c3 100644
--- a/extensions/http-curl/protocols/AgentPrinter.h
+++ b/extensions/http-curl/protocols/AgentPrinter.h
@@ -1,5 +1,4 @@
 /**
- *
  * 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.
@@ -22,11 +21,7 @@
 #include "c2/protocols/RESTProtocol.h"
 #include "c2/C2Protocol.h"
 
-namespace org {
-namespace apache {
-namespace nifi {
-namespace minifi {
-namespace c2 {
+namespace org::apache::nifi::minifi::c2 {
 
 /**
  * Purpose and Justification: Encapsulates printing agent information.
@@ -38,6 +33,8 @@ class AgentPrinter : public HeartbeatJsonSerializer, public HeartbeatReporter {
  public:
   explicit AgentPrinter(const std::string& name, const utils::Identifier& uuid = {});
 
+  EXTENSIONAPI static constexpr const char* Description = "Encapsulates printing agent information.";
+
   /**
    * Initialize agent printer.
    */
@@ -58,8 +55,4 @@ class AgentPrinter : public HeartbeatJsonSerializer, public HeartbeatReporter {
   std::shared_ptr<core::logging::Logger> logger_ = core::logging::LoggerFactory<AgentPrinter>::getLogger();
 };
 
-} /* namespace c2 */
-} /* namespace minifi */
-} /* namespace nifi */
-} /* namespace apache */
-} /* namespace org */
+}  // namespace org::apache::nifi::minifi::c2
diff --git a/extensions/http-curl/protocols/RESTSender.cpp b/extensions/http-curl/protocols/RESTSender.cpp
index fc21a414a..f00bd34e3 100644
--- a/extensions/http-curl/protocols/RESTSender.cpp
+++ b/extensions/http-curl/protocols/RESTSender.cpp
@@ -1,5 +1,4 @@
 /**
- *
  * 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.
@@ -201,6 +200,6 @@ C2Payload RESTSender::sendPayload(const std::string url, const Direction directi
   }
 }
 
-REGISTER_RESOURCE(RESTSender, "Encapsulates the restful protocol that is built upon C2Protocol.");
+REGISTER_RESOURCE(RESTSender, DescriptionOnly);
 
 }  // namespace org::apache::nifi::minifi::c2
diff --git a/extensions/http-curl/protocols/RESTSender.h b/extensions/http-curl/protocols/RESTSender.h
index 31e038315..ff5c9f561 100644
--- a/extensions/http-curl/protocols/RESTSender.h
+++ b/extensions/http-curl/protocols/RESTSender.h
@@ -1,5 +1,4 @@
 /**
- *
  * 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.
@@ -45,6 +44,8 @@ class RESTSender : public RESTProtocol, public C2Protocol {
  public:
   explicit RESTSender(const std::string &name, const utils::Identifier &uuid = utils::Identifier());
 
+  EXTENSIONAPI static constexpr const char* Description = "Encapsulates the restful protocol that is built upon C2Protocol.";
+
   C2Payload consumePayload(const std::string &url, const C2Payload &payload, Direction direction, bool async) override;
 
   C2Payload consumePayload(const C2Payload &payload, Direction direction, bool async) override;
diff --git a/extensions/http-curl/sitetosite/HTTPProtocol.cpp b/extensions/http-curl/sitetosite/HTTPProtocol.cpp
index e1683f4e0..c80bd328c 100644
--- a/extensions/http-curl/sitetosite/HTTPProtocol.cpp
+++ b/extensions/http-curl/sitetosite/HTTPProtocol.cpp
@@ -1,5 +1,4 @@
 /**
- *
  * 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.
@@ -33,11 +32,7 @@
 #include "io/validation.h"
 #include "core/Resource.h"
 
-namespace org {
-namespace apache {
-namespace nifi {
-namespace minifi {
-namespace sitetosite {
+namespace org::apache::nifi::minifi::sitetosite {
 
 std::shared_ptr<utils::IdGenerator> HttpSiteToSiteClient::id_generator_ = utils::IdGenerator::getIdGenerator();
 
@@ -305,10 +300,6 @@ void HttpSiteToSiteClient::deleteTransaction(const utils::Identifier& transactio
   SiteToSiteClient::deleteTransaction(transactionID);
 }
 
-REGISTER_INTERNAL_RESOURCE_AS(HttpSiteToSiteClient, ("HttpSiteToSiteClient", "HttpProtocol"));
+REGISTER_RESOURCE_AS(HttpSiteToSiteClient, InternalResource, ("HttpSiteToSiteClient", "HttpProtocol"));
 
-} /* namespace sitetosite */
-} /* namespace minifi */
-} /* namespace nifi */
-} /* namespace apache */
-} /* namespace org */
+}  // namespace org::apache::nifi::minifi::sitetosite
diff --git a/extensions/http-curl/sitetosite/HTTPProtocol.h b/extensions/http-curl/sitetosite/HTTPProtocol.h
index 118b97940..005f53055 100644
--- a/extensions/http-curl/sitetosite/HTTPProtocol.h
+++ b/extensions/http-curl/sitetosite/HTTPProtocol.h
@@ -1,5 +1,4 @@
 /**
- *
  * 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.
@@ -32,47 +31,36 @@
 #include "sitetosite/Peer.h"
 #include "utils/Id.h"
 
-namespace org {
-namespace apache {
-namespace nifi {
-namespace minifi {
-namespace sitetosite {
+namespace org::apache::nifi::minifi::sitetosite {
 
-/**
- * Site2Site Peer
- */
 typedef struct Site2SitePeerStatus {
   std::string host_;
   int port_;
   bool isSecure_;
 } Site2SitePeerStatus;
 
-// HttpSiteToSiteClient Class
 class HttpSiteToSiteClient : public sitetosite::SiteToSiteClient {
   static constexpr char const* PROTOCOL_VERSION_HEADER = "x-nifi-site-to-site-protocol-version";
 
  public:
-  /*!
-   * Create a new http protocol
-   */
   explicit HttpSiteToSiteClient(const std::string& /*name*/, const utils::Identifier& /*uuid*/ = {})
       : SiteToSiteClient(),
         current_code(UNRECOGNIZED_RESPONSE_CODE) {
     peer_state_ = READY;
   }
 
-  /*!
-   * Create a new http protocol
-   */
   explicit HttpSiteToSiteClient(std::unique_ptr<SiteToSitePeer> peer)
       : SiteToSiteClient(),
         current_code(UNRECOGNIZED_RESPONSE_CODE) {
     peer_ = std::move(peer);
     peer_state_ = READY;
   }
-  // Destructor
   ~HttpSiteToSiteClient() override = default;
 
+  static auto properties() { return std::array<core::Property, 0>{}; }
+  EXTENSIONAPI static constexpr bool SupportsDynamicProperties = false;
+  EXTENSIONAPI static constexpr bool SupportsDynamicRelationships = false;
+
   void setPeer(std::unique_ptr<SiteToSitePeer> peer) override {
     peer_ = std::move(peer);
   }
@@ -175,8 +163,4 @@ class HttpSiteToSiteClient : public sitetosite::SiteToSiteClient {
   static std::shared_ptr<utils::IdGenerator> id_generator_;
 };
 
-} /* namespace sitetosite */
-} /* namespace minifi */
-} /* namespace nifi */
-} /* namespace apache */
-} /* namespace org */
+}  // namespace org::apache::nifi::minifi::sitetosite
diff --git a/extensions/http-curl/tests/AbsoluteTimeoutTest.cpp b/extensions/http-curl/tests/AbsoluteTimeoutTest.cpp
index 9f2c4939a..27b94b2b3 100644
--- a/extensions/http-curl/tests/AbsoluteTimeoutTest.cpp
+++ b/extensions/http-curl/tests/AbsoluteTimeoutTest.cpp
@@ -15,8 +15,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 #undef NDEBUG
+#include <array>
+
 #include "TestBase.h"
 #include "Catch.h"
 #include "tests/TestServer.h"
@@ -38,7 +39,7 @@ int main() {
   auto processor = plan->addProcessor("InvokeHTTP", "InvokeHTTP");
   processor->setProperty("Read Timeout", "1 s");
   processor->setProperty("Remote URL", "http://localhost:" + port);
-  processor->setAutoTerminatedRelationships({{"failure", "d"}});
+  processor->setAutoTerminatedRelationships(std::array{core::Relationship{"failure", "d"}});
 
   plan->runNextProcessor();
 
diff --git a/extensions/http-curl/tests/ControllerServiceIntegrationTests.cpp b/extensions/http-curl/tests/ControllerServiceIntegrationTests.cpp
index 72669b64b..12525b06d 100644
--- a/extensions/http-curl/tests/ControllerServiceIntegrationTests.cpp
+++ b/extensions/http-curl/tests/ControllerServiceIntegrationTests.cpp
@@ -40,8 +40,8 @@
 #include "integration/IntegrationBase.h"
 #include "utils/IntegrationTestUtils.h"
 
-REGISTER_RESOURCE(MockControllerService, "");
-REGISTER_RESOURCE(MockProcessor, "");
+REGISTER_RESOURCE(MockControllerService, ControllerService);
+REGISTER_RESOURCE(MockProcessor, Processor);
 
 void waitToVerifyProcessor() {
   std::this_thread::sleep_for(std::chrono::seconds(2));
diff --git a/extensions/http-curl/tests/HTTPHandlers.h b/extensions/http-curl/tests/HTTPHandlers.h
index 07662569b..cb523bf84 100644
--- a/extensions/http-curl/tests/HTTPHandlers.h
+++ b/extensions/http-curl/tests/HTTPHandlers.h
@@ -40,6 +40,7 @@
 #include "utils/gsl.h"
 #include "agent/build_description.h"
 #include "c2/C2Payload.h"
+#include "core/PropertyBuilder.h"
 #include "properties/Configuration.h"
 #include "range/v3/algorithm/find.hpp"
 
@@ -466,7 +467,7 @@ class HeartbeatHandler : public ServerAwareHandler {
 
         auto group = minifi::BuildDescription{}.getClassDescriptions(str);
         for (const auto& proc : group.processors_) {
-          assert(std::find(classes.begin(), classes.end(), proc.class_name_) != std::end(classes));
+          assert(std::find(classes.begin(), classes.end(), proc.full_name_) != std::end(classes));
           (void)proc;
           found = true;
         }
diff --git a/extensions/http-curl/tests/unit/InvokeHTTPTests.cpp b/extensions/http-curl/tests/unit/InvokeHTTPTests.cpp
index e5df28f91..45938306a 100644
--- a/extensions/http-curl/tests/unit/InvokeHTTPTests.cpp
+++ b/extensions/http-curl/tests/unit/InvokeHTTPTests.cpp
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
+#include <array>
 #include <memory>
 #include <utility>
 #include <string>
@@ -266,7 +266,7 @@ TEST_CASE("HTTPTestsPenalizeNoRetry", "[httptest1]") {
 
   plan->setProperty(invokehttp, InvokeHTTP::Method.getName(), "GET");
   plan->setProperty(invokehttp, InvokeHTTP::URL.getName(), "http://localhost:8681/invalid");
-  invokehttp->setAutoTerminatedRelationships({InvokeHTTP::RelFailure, InvokeHTTP::RelNoRetry, InvokeHTTP::RelResponse, InvokeHTTP::RelRetry});
+  invokehttp->setAutoTerminatedRelationships(std::array{InvokeHTTP::RelFailure, InvokeHTTP::RelNoRetry, InvokeHTTP::RelResponse, InvokeHTTP::RelRetry});
 
   constexpr const char* PENALIZE_LOG_PATTERN = "Penalizing [0-9a-f-]+ for [0-9]+ms at invokehttp";
 
@@ -298,7 +298,7 @@ TEST_CASE("HTTPTestsPutResponseBodyinAttribute", "[httptest1]") {
   plan->setProperty(invokehttp, InvokeHTTP::Method.getName(), "GET");
   plan->setProperty(invokehttp, InvokeHTTP::URL.getName(), TestHTTPServer::URL);
   plan->setProperty(invokehttp, InvokeHTTP::PropPutOutputAttributes.getName(), "http.type");
-  invokehttp->setAutoTerminatedRelationships({InvokeHTTP::RelFailure, InvokeHTTP::RelNoRetry, InvokeHTTP::RelResponse, InvokeHTTP::RelRetry});
+  invokehttp->setAutoTerminatedRelationships(std::array{InvokeHTTP::RelFailure, InvokeHTTP::RelNoRetry, InvokeHTTP::RelResponse, InvokeHTTP::RelRetry});
   testController.runSession(plan);
 
   REQUIRE(LogTestController::getInstance().contains("Adding http response body to flow file attribute http.type"));
@@ -316,7 +316,7 @@ TEST_CASE("InvokeHTTP fails with when flow contains invalid attribute names in H
   invokehttp->setProperty(InvokeHTTP::URL, TestHTTPServer::URL);
   invokehttp->setProperty(InvokeHTTP::InvalidHTTPHeaderFieldHandlingStrategy, "fail");
   invokehttp->setProperty(InvokeHTTP::AttributesToSend, ".*");
-  invokehttp->setAutoTerminatedRelationships({InvokeHTTP::RelNoRetry, InvokeHTTP::Success, InvokeHTTP::RelResponse, InvokeHTTP::RelRetry});
+  invokehttp->setAutoTerminatedRelationships(std::array{InvokeHTTP::RelNoRetry, InvokeHTTP::Success, InvokeHTTP::RelResponse, InvokeHTTP::RelRetry});
   const auto result = test_controller.trigger("data", {{"invalid header", "value"}});
   auto file_contents = result.at(InvokeHTTP::RelFailure);
   REQUIRE(file_contents.size() == 1);
@@ -336,7 +336,7 @@ TEST_CASE("InvokeHTTP succeeds when the flow file contains an attribute that wou
   invokehttp->setProperty(InvokeHTTP::URL, TestHTTPServer::URL);
   invokehttp->setProperty(InvokeHTTP::InvalidHTTPHeaderFieldHandlingStrategy, "fail");
   invokehttp->setProperty(InvokeHTTP::AttributesToSend, "valid.*");
-  invokehttp->setAutoTerminatedRelationships({InvokeHTTP::RelNoRetry, InvokeHTTP::Success, InvokeHTTP::RelResponse, InvokeHTTP::RelRetry});
+  invokehttp->setAutoTerminatedRelationships(std::array{InvokeHTTP::RelNoRetry, InvokeHTTP::Success, InvokeHTTP::RelResponse, InvokeHTTP::RelRetry});
   const auto result = test_controller.trigger("data", {{"invalid header", "value"}, {"valid-header", "value2"}});
   REQUIRE(result.at(InvokeHTTP::RelFailure).empty());
   const auto& success_contents = result.at(InvokeHTTP::Success);
@@ -357,7 +357,7 @@ TEST_CASE("InvokeHTTP replaces invalid characters of attributes", "[httptest1]")
   invokehttp->setProperty(InvokeHTTP::Method, "GET");
   invokehttp->setProperty(InvokeHTTP::URL, TestHTTPServer::URL);
   invokehttp->setProperty(InvokeHTTP::AttributesToSend, ".*");
-  invokehttp->setAutoTerminatedRelationships({InvokeHTTP::RelNoRetry, InvokeHTTP::RelFailure, InvokeHTTP::RelResponse, InvokeHTTP::RelRetry});
+  invokehttp->setAutoTerminatedRelationships(std::array{InvokeHTTP::RelNoRetry, InvokeHTTP::RelFailure, InvokeHTTP::RelResponse, InvokeHTTP::RelRetry});
   const auto result = test_controller.trigger("data", {{"invalid header", "value"}, {"", "value2"}});
   auto file_contents = result.at(InvokeHTTP::Success);
   REQUIRE(file_contents.size() == 1);
@@ -379,7 +379,7 @@ TEST_CASE("InvokeHTTP drops invalid attributes from HTTP headers", "[httptest1]"
   invokehttp->setProperty(InvokeHTTP::URL, TestHTTPServer::URL);
   invokehttp->setProperty(InvokeHTTP::InvalidHTTPHeaderFieldHandlingStrategy, "drop");
   invokehttp->setProperty(InvokeHTTP::AttributesToSend, ".*");
-  invokehttp->setAutoTerminatedRelationships({InvokeHTTP::RelNoRetry, InvokeHTTP::RelFailure, InvokeHTTP::RelResponse, InvokeHTTP::RelRetry});
+  invokehttp->setAutoTerminatedRelationships(std::array{InvokeHTTP::RelNoRetry, InvokeHTTP::RelFailure, InvokeHTTP::RelResponse, InvokeHTTP::RelRetry});
   const auto result = test_controller.trigger("data", {{"legit-header", "value1"}, {"invalid header", "value2"}});
   auto file_contents = result.at(InvokeHTTP::Success);
   REQUIRE(file_contents.size() == 1);
@@ -401,7 +401,7 @@ TEST_CASE("InvokeHTTP empty Attributes to Send means no attributes are sent", "[
   invokehttp->setProperty(InvokeHTTP::URL, TestHTTPServer::URL);
   invokehttp->setProperty(InvokeHTTP::InvalidHTTPHeaderFieldHandlingStrategy, "drop");
   invokehttp->setProperty(InvokeHTTP::AttributesToSend, "");
-  invokehttp->setAutoTerminatedRelationships({InvokeHTTP::RelNoRetry, InvokeHTTP::RelFailure, InvokeHTTP::RelResponse, InvokeHTTP::RelRetry});
+  invokehttp->setAutoTerminatedRelationships(std::array{InvokeHTTP::RelNoRetry, InvokeHTTP::RelFailure, InvokeHTTP::RelResponse, InvokeHTTP::RelRetry});
   const auto result = test_controller.trigger("data", {{"legit-header", "value1"}, {"invalid header", "value2"}});
   auto file_contents = result.at(InvokeHTTP::Success);
   REQUIRE(file_contents.size() == 1);
@@ -423,7 +423,7 @@ TEST_CASE("InvokeHTTP Attributes to Send uses full string matching, not substrin
   invokehttp->setProperty(InvokeHTTP::URL, TestHTTPServer::URL);
   invokehttp->setProperty(InvokeHTTP::InvalidHTTPHeaderFieldHandlingStrategy, "drop");
   invokehttp->setProperty(InvokeHTTP::AttributesToSend, "he.*er");
-  invokehttp->setAutoTerminatedRelationships({InvokeHTTP::RelNoRetry, InvokeHTTP::RelFailure, InvokeHTTP::RelResponse, InvokeHTTP::RelRetry});
+  invokehttp->setAutoTerminatedRelationships(std::array{InvokeHTTP::RelNoRetry, InvokeHTTP::RelFailure, InvokeHTTP::RelResponse, InvokeHTTP::RelRetry});
   const auto result = test_controller.trigger("data", {{"header1", "value1"}, {"header", "value2"}});
   auto file_contents = result.at(InvokeHTTP::Success);
   REQUIRE(file_contents.size() == 1);
diff --git a/extensions/jni/ExecuteJavaControllerService.cpp b/extensions/jni/ExecuteJavaControllerService.cpp
index bf448e8d4..c5e70bb29 100644
--- a/extensions/jni/ExecuteJavaControllerService.cpp
+++ b/extensions/jni/ExecuteJavaControllerService.cpp
@@ -27,7 +27,6 @@
 #include <iostream>
 #include <iterator>
 #include <map>
-#include <set>
 #include <string>
 #include <utility>
 #include <vector>
@@ -35,6 +34,7 @@
 #include "core/FlowFile.h"
 #include "core/logging/Logger.h"
 #include "core/ProcessContext.h"
+#include "core/PropertyBuilder.h"
 #include "core/Relationship.h"
 #include "core/Resource.h"
 #include "ResourceClaim.h"
@@ -49,19 +49,15 @@ namespace minifi {
 namespace jni {
 namespace controllers {
 
-core::Property ExecuteJavaControllerService::NiFiControllerService(
+const core::Property ExecuteJavaControllerService::NiFiControllerService(
     core::PropertyBuilder::createProperty("NiFi Controller Service")->withDescription("Name of NiFi Controller Service to load and run")->isRequired(true)->withDefaultValue<std::string>("")->build());
 
-const char *ExecuteJavaControllerService::ProcessorName = "ExecuteJavaControllerService";
-
 void ExecuteJavaControllerService::initialize() {
   logger_->log_info("Initializing ExecuteJavaControllerService");
-  // Set the supported properties
+
   std::string existingValue;
   getProperty(NiFiControllerService.getName(), existingValue);
-  std::set<core::Property> properties;
-  properties.insert(NiFiControllerService);
-  setSupportedProperties(properties);
+  setSupportedProperties(std::array{NiFiControllerService});
   setAcceptAllProperties();
 
   if (!existingValue.empty()) {
@@ -106,7 +102,7 @@ void ExecuteJavaControllerService::onEnable() {
   }
 }
 
-REGISTER_RESOURCE(ExecuteJavaControllerService, "ExecuteJavaClass runs NiFi Controller services given a provided system path ");
+REGISTER_RESOURCE(ExecuteJavaControllerService, ControllerService);
 
 } /* namespace controllers */
 } /* namespace jni */
diff --git a/extensions/jni/ExecuteJavaControllerService.h b/extensions/jni/ExecuteJavaControllerService.h
index 89ca9b27c..f4f553d08 100644
--- a/extensions/jni/ExecuteJavaControllerService.h
+++ b/extensions/jni/ExecuteJavaControllerService.h
@@ -1,6 +1,4 @@
 /**
- * ExecuteJavaClass class declaration
- *
  * 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.
@@ -35,6 +33,7 @@
 #include "utils/Id.h"
 #include "jvm/NarClassLoader.h"
 #include "ClassRegistrar.h"
+
 namespace org {
 namespace apache {
 namespace nifi {
@@ -54,25 +53,19 @@ namespace controllers {
  */
 class ExecuteJavaControllerService : public ConfigurationContext, public std::enable_shared_from_this<ConfigurationContext> {
  public:
-  // Constructor
-  /*!
-   * Create a new processor
-   */
   explicit ExecuteJavaControllerService(const std::string& name, const utils::Identifier& uuid = {})
       : ConfigurationContext(name, uuid) {
   }
-  // Destructor
-  virtual ~ExecuteJavaControllerService();
-  // Processor Name
-  static const char *ProcessorName;
-  static core::Property NiFiControllerService;
-  // Supported Relationships
+  ~ExecuteJavaControllerService() override;
+
+  EXTENSIONAPI static constexpr const char* Description = "ExecuteJavaClass runs NiFi Controller services given a provided system path";
+  EXTENSIONAPI static const core::Property NiFiControllerService;
+  static auto properties() { return std::array{NiFiControllerService}; }
+  EXTENSIONAPI static constexpr bool SupportsDynamicProperties = true;
+  ADD_COMMON_VIRTUAL_FUNCTIONS_FOR_CONTROLLER_SERVICES
 
   void onEnable() override;
   void initialize() override;
-  bool supportsDynamicProperties() override {
-    return true;
-  }
 
   void yield() override {
   }
diff --git a/extensions/jni/ExecuteJavaProcessor.cpp b/extensions/jni/ExecuteJavaProcessor.cpp
index 5f16a75ef..c220db915 100644
--- a/extensions/jni/ExecuteJavaProcessor.cpp
+++ b/extensions/jni/ExecuteJavaProcessor.cpp
@@ -27,7 +27,6 @@
 #include <iostream>
 #include <iterator>
 #include <map>
-#include <set>
 #include <string>
 #include <utility>
 #include <vector>
@@ -35,6 +34,7 @@
 #include "core/FlowFile.h"
 #include "core/logging/Logger.h"
 #include "core/ProcessContext.h"
+#include "core/PropertyBuilder.h"
 #include "core/Relationship.h"
 #include "core/Resource.h"
 #include "ResourceClaim.h"
@@ -58,21 +58,13 @@ core::Property ExecuteJavaProcessor::NiFiProcessor(core::PropertyBuilder::create
     ->withDescription("Name of NiFi processor to load and run")
     ->isRequired(true)->withDefaultValue<std::string>("")->build());
 
-const char *ExecuteJavaProcessor::ProcessorName = "ExecuteJavaClass";
-
 core::Relationship ExecuteJavaProcessor::Success("success", "All files are routed to success");
+
 void ExecuteJavaProcessor::initialize() {
   logger_->log_info("Initializing ExecuteJavaClass");
-  // Set the supported properties
-  std::set<core::Property> properties;
-  properties.insert(JVMControllerService);
-  properties.insert(NiFiProcessor);
-  setSupportedProperties(properties);
+  setSupportedProperties(properties());
   setAcceptAllProperties();
-  // Set the supported relationships
-  std::set<core::Relationship> relationships;
-  relationships.insert(Success);
-  setSupportedRelationships(relationships);
+  setSupportedRelationships(relationships());
 }
 
 void ExecuteJavaProcessor::onSchedule(const std::shared_ptr<core::ProcessContext>& context, const std::shared_ptr<core::ProcessSessionFactory>& /*sessionFactory*/) {
@@ -233,7 +225,7 @@ void ExecuteJavaProcessor::onTrigger(const std::shared_ptr<core::ProcessContext>
   // do nothing.
 }
 
-REGISTER_RESOURCE_AS(ExecuteJavaProcessor, "ExecuteJavaClass runs NiFi processors given a provided system path ", ("ExecuteJavaClass"));
+REGISTER_RESOURCE_AS(ExecuteJavaProcessor, Processor, ("ExecuteJavaClass"));
 
 } /* namespace processors */
 } /* namespace jni */
diff --git a/extensions/jni/ExecuteJavaProcessor.h b/extensions/jni/ExecuteJavaProcessor.h
index af17bb794..b334d2957 100644
--- a/extensions/jni/ExecuteJavaProcessor.h
+++ b/extensions/jni/ExecuteJavaProcessor.h
@@ -52,29 +52,27 @@ namespace processors {
  */
 class ExecuteJavaProcessor : public core::Processor {
  public:
-  // Constructor
-  /*!
-   * Create a new processor
-   */
   explicit ExecuteJavaProcessor(const std::string& name, const utils::Identifier& uuid = {})
       : Processor(name, uuid) {
   }
-  // Destructor
   virtual ~ExecuteJavaProcessor();
-  // Processor Name
-  static const char *ProcessorName;
+
+  EXTENSIONAPI static constexpr const char* Description = "ExecuteJavaClass runs NiFi processors given a provided system path ";
   static core::Property JVMControllerService;
   static core::Property NiFiProcessor;
-  // Supported Relationships
+  static auto properties() { return std::array{JVMControllerService, NiFiProcessor}; }
   static core::Relationship Success;
+  static auto relationships() { return std::array{Success}; }
+  EXTENSIONAPI static constexpr bool SupportsDynamicProperties = true;
+  EXTENSIONAPI static constexpr bool SupportsDynamicRelationships = false;
+  EXTENSIONAPI static constexpr core::annotation::Input InputRequirement = core::annotation::Input::INPUT_ALLOWED;
+  EXTENSIONAPI static constexpr bool IsSingleThreaded = false;
+  ADD_COMMON_VIRTUAL_FUNCTIONS_FOR_PROCESSORS
 
   void onTrigger(const std::shared_ptr<core::ProcessContext> &context, const std::shared_ptr<core::ProcessSessionFactory> &sessionFactory) override;
   void onTrigger(const std::shared_ptr<core::ProcessContext> &context, const std::shared_ptr<core::ProcessSession> &session) override;
   void initialize() override;
   void onSchedule(const std::shared_ptr<core::ProcessContext> &context, const std::shared_ptr<core::ProcessSessionFactory> &sessionFactory) override;
-  bool supportsDynamicProperties() override {
-    return true;
-  }
 
  protected:
   static JavaSignatures &getLoggerSignatures() {
diff --git a/extensions/jni/JVMCreator.cpp b/extensions/jni/JVMCreator.cpp
index 239fc96df..b65c2807d 100644
--- a/extensions/jni/JVMCreator.cpp
+++ b/extensions/jni/JVMCreator.cpp
@@ -1,5 +1,4 @@
 /**
- *
  * 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.
@@ -25,11 +24,7 @@
 #include <dirent.h>
 #endif
 
-namespace org {
-namespace apache {
-namespace nifi {
-namespace minifi {
-namespace jni {
+namespace org::apache::nifi::minifi::jni {
 
 #ifndef S_ISDIR
 #define S_ISDIR(mode)  (((mode) & S_IFMT) == S_IFDIR)
@@ -41,10 +36,6 @@ namespace jni {
 #endif
 JVMCreator::~JVMCreator() = default;
 
-REGISTER_INTERNAL_RESOURCE(JVMCreator);
+REGISTER_RESOURCE(JVMCreator, InternalResource);
 
-} /* namespace jni */
-} /* namespace minifi */
-} /* namespace nifi */
-} /* namespace apache */
-} /* namespace org */
+}  // namespace org::apache::nifi::minifi::jni
diff --git a/extensions/jni/JVMCreator.h b/extensions/jni/JVMCreator.h
index 9979a1806..dbc1401fb 100644
--- a/extensions/jni/JVMCreator.h
+++ b/extensions/jni/JVMCreator.h
@@ -1,5 +1,4 @@
 /**
- *
  * 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.
@@ -28,11 +27,7 @@
 #include "core/logging/LoggerConfiguration.h"
 #include "properties/Configuration.h"
 
-namespace org {
-namespace apache {
-namespace nifi {
-namespace minifi {
-namespace jni {
+namespace org::apache::nifi::minifi::jni {
 
 /**
  * Can be used to load the JVM from NiFi properties.
@@ -43,7 +38,11 @@ class JVMCreator : public minifi::core::CoreComponent {
       : minifi::core::CoreComponent(name, uuid) {
   }
 
-  virtual ~JVMCreator();
+  ~JVMCreator() override;
+
+  static auto properties() { return std::array<core::Property, 0>{}; }
+  EXTENSIONAPI static constexpr bool SupportsDynamicProperties = false;
+  EXTENSIONAPI static constexpr bool SupportsDynamicRelationships = false;
 
   void configure(const std::vector<std::string> &jarFileListings) {
     std::vector<std::string> pathOrFiles;
@@ -100,8 +99,4 @@ class JVMCreator : public minifi::core::CoreComponent {
   std::shared_ptr<core::logging::Logger> logger_ = core::logging::LoggerFactory<JVMCreator>::getLogger();
 };
 
-} /* namespace jni */
-} /* namespace minifi */
-} /* namespace nifi */
-} /* namespace apache */
-} /* namespace org */
+}  // namespace org::apache::nifi::minifi::jni
diff --git a/extensions/jni/jvm/JavaControllerService.cpp b/extensions/jni/jvm/JavaControllerService.cpp
index 933e27c99..cb823a30c 100644
--- a/extensions/jni/jvm/JavaControllerService.cpp
+++ b/extensions/jni/jvm/JavaControllerService.cpp
@@ -22,8 +22,7 @@
 #include <memory>
 #include <algorithm>
 #include <iterator>
-#include <set>
-#include "core/Property.h"
+#include "core/PropertyBuilder.h"
 #include "core/Resource.h"
 #include "io/validation.h"
 #include "utils/StringUtils.h"
@@ -45,17 +44,13 @@ namespace controllers {
 #define W_OK    2       /* Test for write permission.  */
 #define F_OK    0       /* Test for existence.  */
 #endif
-static core::Property NarDirectory;
-static core::Property NarDeploymentDirectory;
-static core::Property NarDocumentDirectory;
-
-core::Property JavaControllerService::NarDirectory(
+const core::Property JavaControllerService::NarDirectory(
     core::PropertyBuilder::createProperty("Nar Directory")->withDescription("Directory containing the nars to deploy")->isRequired(true)->supportsExpressionLanguage(false)->build());
 
-core::Property JavaControllerService::NarDeploymentDirectory(
+const core::Property JavaControllerService::NarDeploymentDirectory(
     core::PropertyBuilder::createProperty("Nar Deployment Directory")->withDescription("Directory in which nars will be deployed")->isRequired(true)->supportsExpressionLanguage(false)->build());
 
-core::Property JavaControllerService::NarDocumentDirectory(
+const core::Property JavaControllerService::NarDocumentDirectory(
     core::PropertyBuilder::createProperty("Nar Document Directory")->withDescription("Directory in which documents will be deployed")->isRequired(true)->supportsExpressionLanguage(false)->build());
 
 void JavaControllerService::initialize() {
@@ -66,12 +61,7 @@ void JavaControllerService::initialize() {
 
   ControllerService::initialize();
 
-  std::set<core::Property> supportedProperties;
-  supportedProperties.insert(NarDirectory);
-  supportedProperties.insert(NarDeploymentDirectory);
-  supportedProperties.insert(NarDocumentDirectory);
-
-  setSupportedProperties(supportedProperties);
+  setSupportedProperties(properties());
 
   initialized_ = true;
 }
@@ -109,7 +99,7 @@ void JavaControllerService::onEnable() {
   nar_loader_ = std::unique_ptr<NarClassLoader>(new NarClassLoader(shared_from_this(), narClassLoaderClazz, nardir, narscratch, nardocs));
 }
 
-REGISTER_RESOURCE(JavaControllerService, "Allows specification of nars to be used within referenced processors. ");
+REGISTER_RESOURCE(JavaControllerService, ControllerService);
 
 } /* namespace controllers */
 } /* namespace jni */
diff --git a/extensions/jni/jvm/JavaControllerService.h b/extensions/jni/jvm/JavaControllerService.h
index 9b8c27817..9293c434d 100644
--- a/extensions/jni/jvm/JavaControllerService.h
+++ b/extensions/jni/jvm/JavaControllerService.h
@@ -42,7 +42,6 @@ namespace controllers {
 /**
  * Purpose and Justification: Java Controller Service is intended to be used either within the flow or
  * based on a static load in JVM Creator. The static load simply loads via minifi properties.
- *
  */
 class JavaControllerService : public core::controller::ControllerService, public std::enable_shared_from_this<JavaControllerService>, public JavaServicer {
  public:
@@ -56,9 +55,21 @@ class JavaControllerService : public core::controller::ControllerService, public
     initialize();
   }
 
-  static core::Property NarDirectory;
-  static core::Property NarDeploymentDirectory;
-  static core::Property NarDocumentDirectory;
+  EXTENSIONAPI static constexpr const char* Description = "Allows specification of nars to be used within referenced processors.";
+
+  EXTENSIONAPI static const core::Property NarDirectory;
+  EXTENSIONAPI static const core::Property NarDeploymentDirectory;
+  EXTENSIONAPI static const core::Property NarDocumentDirectory;
+  static auto properties() {
+    return std::array{
+      NarDirectory,
+      NarDeploymentDirectory,
+      NarDocumentDirectory
+    };
+  }
+
+  EXTENSIONAPI static constexpr bool SupportsDynamicProperties = false;
+  ADD_COMMON_VIRTUAL_FUNCTIONS_FOR_CONTROLLER_SERVICES
 
   void initialize() override;
 
@@ -92,12 +103,11 @@ class JavaControllerService : public core::controller::ControllerService, public
   JNIEnv *attach() override {
     return loader->attach();
   }
+
   void detach() override {
       loader->detach();
     }
 
-
-
   jobject getClassLoader() override {
     return loader->getClassLoader();
   }
diff --git a/extensions/jni/jvm/JniConfigurationContext.h b/extensions/jni/jvm/JniConfigurationContext.h
index add88e9e1..3a9850f08 100644
--- a/extensions/jni/jvm/JniConfigurationContext.h
+++ b/extensions/jni/jvm/JniConfigurationContext.h
@@ -1,5 +1,4 @@
 /**
- *
  * 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.
@@ -31,15 +30,11 @@ namespace jni {
 
 class ConfigurationContext : public core::controller::ControllerService {
  public:
-  // Constructor
-  /*!
-   * Create a new processor
-   */
   explicit ConfigurationContext(const std::string& name, const utils::Identifier& uuid = {})
       : core::controller::ControllerService(name, uuid) {
   }
 
-  virtual ~ConfigurationContext() = default;
+  ~ConfigurationContext() override = default;
   virtual jobject getClassInstance() = 0;
 };
 
diff --git a/extensions/jni/jvm/NarClassLoader.h b/extensions/jni/jvm/NarClassLoader.h
index 4f72a9b8d..96a80d2c4 100644
--- a/extensions/jni/jvm/NarClassLoader.h
+++ b/extensions/jni/jvm/NarClassLoader.h
@@ -32,6 +32,7 @@
 #include "JniBundle.h"
 #include "../JavaException.h"
 #include "agent/agent_docs.h"
+#include "core/PropertyBuilder.h"
 
 namespace org {
 namespace apache {
@@ -224,11 +225,11 @@ class NarClassLoader {
     for (size_t i = 0; i < size; i++) {
       JniBundle bundle = getBundle(list_class, env, obj, i);
       for (const auto &cd : bundle.getDescriptions()) {
-        auto lastOfIdx = cd.class_name_.find_last_of(".");
+        auto lastOfIdx = cd.full_name_.find_last_of(".");
         if (lastOfIdx != std::string::npos) {
           lastOfIdx++;  // if a value is found, increment to move beyond the .
-          int nameLength = cd.class_name_.length() - lastOfIdx;
-          const auto processorName = cd.class_name_.substr(lastOfIdx, nameLength);
+          int nameLength = cd.full_name_.length() - lastOfIdx;
+          const auto processorName = cd.full_name_.substr(lastOfIdx, nameLength);
           if (!core::ClassLoader::getDefaultClassLoader().getGroupForClass(processorName)) {
             minifi::ExternalBuildDescription::addExternalComponent(bundle.getDetails(), cd);
           }
@@ -304,7 +305,7 @@ class NarClassLoader {
     if (component != nullptr) {
       auto type = getStringMethod("getType", jni_component_clazz, env, component);
       auto isControllerService = getBoolmethod("isControllerService", jni_component_clazz, env, component);
-      ClassDescription description(type);
+      ClassDescription description{.full_name_ = type};
       {
         jmethodID getDescriptorMethod = env->GetMethodID(jni_component_clazz, "getDescriptors", "()Ljava/util/List;");
 
@@ -331,12 +332,12 @@ class NarClassLoader {
 
               builder = builder->isRequired(getBoolmethod("isRequired", property_descriptor_clazz, env, propertyDescriptorObj));
               core::Property prop(builder->build());
-              description.class_properties_.insert(std::make_pair(prop.getName(), prop));
+              description.class_properties_.push_back(prop);
             }
           }
         }
       }
-      description.is_controller_service_ = isControllerService;
+      description.type_ = isControllerService ? ResourceType::ControllerService : ResourceType::Processor;
       jmethodID getRelationshipsMethod = env->GetMethodID(jni_component_clazz, "getRelationships", "()Ljava/util/List;");
       ThrowIf(env);
       jobject relationships = env->CallObjectMethod(component, getRelationshipsMethod);
@@ -363,13 +364,13 @@ class NarClassLoader {
       description.dynamic_relationships_ = getBoolmethod("getDynamicRelationshipsSupported", jni_component_clazz, env, component);
       description.dynamic_properties_ = getBoolmethod("getDynamicPropertiesSupported", jni_component_clazz, env, component);
 
-      AgentDocs::putDescription(type, classDescription);
+      description.description_ = classDescription;
 
       return description;
     }
     // assuming we have the bundle, we need to get the coordinate.
 
-    return ClassDescription("unknown");
+    return ClassDescription{.full_name_ = "unknown"};
   }
 
   struct BundleDetails getCoordinateDetails(JNIEnv *env, jclass jni_bundle_clazz, jobject bundle) {
diff --git a/extensions/kubernetes/controllerservice/KubernetesControllerService.cpp b/extensions/kubernetes/controllerservice/KubernetesControllerService.cpp
index 18d8fbb8e..d022f96a5 100644
--- a/extensions/kubernetes/controllerservice/KubernetesControllerService.cpp
+++ b/extensions/kubernetes/controllerservice/KubernetesControllerService.cpp
@@ -26,6 +26,7 @@ extern "C" {
 #include "api/CoreV1API.h"
 }
 
+#include "core/PropertyBuilder.h"
 #include "core/Resource.h"
 #include "core/logging/LoggerConfiguration.h"
 #include "Exception.h"
@@ -103,7 +104,7 @@ void KubernetesControllerService::initialize() {
   if (initialized_) { return; }
 
   ControllerService::initialize();
-  setSupportedProperties({NamespaceFilter, PodNameFilter, ContainerNameFilter});
+  setSupportedProperties(properties());
   initialized_ = true;
 }
 
@@ -207,6 +208,6 @@ bool KubernetesControllerService::matchesRegexFilters(const std::string& name_sp
       matchesFilter(container_name, container_name_filter_);
 }
 
-REGISTER_RESOURCE(KubernetesControllerService, "Controller service that provides access to the Kubernetes API");
+REGISTER_RESOURCE(KubernetesControllerService, ControllerService);
 
 }  // namespace org::apache::nifi::minifi::controllers
diff --git a/extensions/kubernetes/controllerservice/KubernetesControllerService.h b/extensions/kubernetes/controllerservice/KubernetesControllerService.h
index 51bbc8777..252585935 100644
--- a/extensions/kubernetes/controllerservice/KubernetesControllerService.h
+++ b/extensions/kubernetes/controllerservice/KubernetesControllerService.h
@@ -30,12 +30,24 @@ namespace org::apache::nifi::minifi::controllers {
 
 class KubernetesControllerService : public AttributeProviderService {
  public:
+  explicit KubernetesControllerService(const std::string& name, const utils::Identifier& uuid = {});
+  KubernetesControllerService(const std::string& name, const std::shared_ptr<Configure>& configuration);
+
+  EXTENSIONAPI static constexpr const char* Description = "Controller service that provides access to the Kubernetes API";
+
   EXTENSIONAPI static const core::Property NamespaceFilter;
   EXTENSIONAPI static const core::Property PodNameFilter;
   EXTENSIONAPI static const core::Property ContainerNameFilter;
+  static auto properties() {
+    return std::array{
+      NamespaceFilter,
+      PodNameFilter,
+      ContainerNameFilter
+    };
+  }
 
-  explicit KubernetesControllerService(const std::string& name, const utils::Identifier& uuid = {});
-  KubernetesControllerService(const std::string& name, const std::shared_ptr<Configure>& configuration);
+  EXTENSIONAPI static constexpr bool SupportsDynamicProperties = false;
+  ADD_COMMON_VIRTUAL_FUNCTIONS_FOR_CONTROLLER_SERVICES
 
   void initialize() final;
   void onEnable() override;
diff --git a/extensions/libarchive/ArchiveStreamProvider.cpp b/extensions/libarchive/ArchiveStreamProvider.cpp
index 776a10490..12be6d18b 100644
--- a/extensions/libarchive/ArchiveStreamProvider.cpp
+++ b/extensions/libarchive/ArchiveStreamProvider.cpp
@@ -1,5 +1,4 @@
 /**
- *
  * 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.
@@ -27,6 +26,11 @@ namespace org::apache::nifi::minifi::io {
 class ArchiveStreamProviderImpl : public ArchiveStreamProvider {
  public:
   using ArchiveStreamProvider::ArchiveStreamProvider;
+
+  static auto properties() { return std::array<core::Property, 0>{}; }
+  EXTENSIONAPI static constexpr bool SupportsDynamicProperties = false;
+  EXTENSIONAPI static constexpr bool SupportsDynamicRelationships = false;
+
   std::unique_ptr<WriteArchiveStream> createWriteStream(int compress_level, const std::string& compress_format,
                                                         std::shared_ptr<OutputStream> sink, std::shared_ptr<core::logging::Logger> logger) override {
     CompressionFormat format = CompressionFormat::parse(compress_format.c_str(), CompressionFormat{});
@@ -44,6 +48,6 @@ class ArchiveStreamProviderImpl : public ArchiveStreamProvider {
   }
 };
 
-REGISTER_INTERNAL_RESOURCE_AS(ArchiveStreamProviderImpl, ("ArchiveStreamProvider"));
+REGISTER_RESOURCE_AS(ArchiveStreamProviderImpl, InternalResource, ("ArchiveStreamProvider"));
 
 }  // namespace org::apache::nifi::minifi::io
diff --git a/extensions/libarchive/BinFiles.cpp b/extensions/libarchive/BinFiles.cpp
index 9ba49e4df..26a2fdb1a 100644
--- a/extensions/libarchive/BinFiles.cpp
+++ b/extensions/libarchive/BinFiles.cpp
@@ -1,7 +1,4 @@
 /**
- * @file BinFiles.cpp
- * BinFiles class implementation
- *
  * 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.
@@ -32,43 +29,10 @@
 #include "core/ProcessSession.h"
 #include "core/Resource.h"
 
-namespace org {
-namespace apache {
-namespace nifi {
-namespace minifi {
-namespace processors {
+namespace org::apache::nifi::minifi::processors {
+
+const core::Relationship BinFiles::Self("__self__", "Marks the FlowFile to be owned by this processor");
 
-core::Property BinFiles::MinSize(
-    core::PropertyBuilder::createProperty("Minimum Group Size")
-    ->withDescription("The minimum size of for the bundle")
-    ->withDefaultValue<uint64_t>(0)->build());
-core::Property BinFiles::MaxSize(
-    core::PropertyBuilder::createProperty("Maximum Group Size")
-    ->withDescription("The maximum size for the bundle. If not specified, there is no maximum.")
-    ->withType(core::StandardValidators::get().UNSIGNED_LONG_VALIDATOR)->build());
-core::Property BinFiles::MinEntries(
-    core::PropertyBuilder::createProperty("Minimum Number of Entries")
-    ->withDescription("The minimum number of files to include in a bundle")
-    ->withDefaultValue<uint32_t>(1)->build());
-core::Property BinFiles::MaxEntries(
-    core::PropertyBuilder::createProperty("Maximum Number of Entries")
-    ->withDescription("The maximum number of files to include in a bundle. If not specified, there is no maximum.")
-    ->withType(core::StandardValidators::get().UNSIGNED_INT_VALIDATOR)->build());
-core::Property BinFiles::MaxBinAge(
-    core::PropertyBuilder::createProperty("Max Bin Age")
-    ->withDescription("The maximum age of a Bin that will trigger a Bin to be complete. Expected format is <duration> <time unit>")
-    ->withType(core::StandardValidators::get().TIME_PERIOD_VALIDATOR)->build());
-core::Property BinFiles::MaxBinCount(
-    core::PropertyBuilder::createProperty("Maximum number of Bins")
-    ->withDescription("Specifies the maximum number of bins that can be held in memory at any one time")
-    ->withDefaultValue<uint32_t>(100)->build());
-core::Property BinFiles::BatchSize(
-    core::PropertyBuilder::createProperty("Batch Size")
-    ->withDescription("Maximum number of FlowFiles processed in a single session")
-    ->withDefaultValue<uint32_t>(1)->build());
-core::Relationship BinFiles::Original("original", "The FlowFiles that were used to create the bundle");
-core::Relationship BinFiles::Failure("failure", "If the bundle cannot be created, all FlowFiles that would have been used to create the bundle will be transferred to failure");
-core::Relationship BinFiles::Self("__self__", "Marks the FlowFile to be owned by this processor");
 const char *BinFiles::FRAGMENT_COUNT_ATTRIBUTE = "fragment.count";
 const char *BinFiles::FRAGMENT_ID_ATTRIBUTE = "fragment.identifier";
 const char *BinFiles::FRAGMENT_INDEX_ATTRIBUTE = "fragment.index";
@@ -79,21 +43,8 @@ const char *BinFiles::SEGMENT_ORIGINAL_FILENAME = "segment.original.filename";
 const char *BinFiles::TAR_PERMISSIONS_ATTRIBUTE = "tar.permissions";
 
 void BinFiles::initialize() {
-  // Set the supported properties
-  std::set<core::Property> properties;
-  properties.insert(MinSize);
-  properties.insert(MaxSize);
-  properties.insert(MinEntries);
-  properties.insert(MaxEntries);
-  properties.insert(MaxBinAge);
-  properties.insert(MaxBinCount);
-  properties.insert(BatchSize);
-  setSupportedProperties(properties);
-  // Set the supported relationships
-  std::set<core::Relationship> relationships;
-  relationships.insert(Original);
-  relationships.insert(Failure);
-  setSupportedRelationships(relationships);
+  setSupportedProperties(properties());
+  setSupportedRelationships(relationships());
 }
 
 void BinFiles::onSchedule(core::ProcessContext *context, core::ProcessSessionFactory* /*sessionFactory*/) {
@@ -356,10 +307,4 @@ std::set<core::Connectable*> BinFiles::getOutGoingConnections(const std::string
   return result;
 }
 
-REGISTER_RESOURCE(BinFiles, "Bins flow files into buckets based on the number of entries or size of entries");
-
-} /* namespace processors */
-} /* namespace minifi */
-} /* namespace nifi */
-} /* namespace apache */
-} /* namespace org */
+}  // namespace org::apache::nifi::minifi::processors
diff --git a/extensions/libarchive/BinFiles.h b/extensions/libarchive/BinFiles.h
index 6ca449794..d3ede9e2a 100644
--- a/extensions/libarchive/BinFiles.h
+++ b/extensions/libarchive/BinFiles.h
@@ -1,7 +1,4 @@
 /**
- * @file BinFiles.h
- * BinFiles class declaration
- *
  * 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.
@@ -36,11 +33,7 @@
 #include "utils/Export.h"
 #include "core/FlowFileStore.h"
 
-namespace org {
-namespace apache {
-namespace nifi {
-namespace minifi {
-namespace processors {
+namespace org::apache::nifi::minifi::processors {
 
 // Bin Class
 class Bin {
@@ -191,29 +184,50 @@ class BinManager {
   std::shared_ptr<core::logging::Logger> logger_{core::logging::LoggerFactory<BinManager>::getLogger()};
 };
 
-// BinFiles Class
 class BinFiles : public core::Processor {
  protected:
-  static core::Relationship Self;
+  static const core::Relationship Self;
 
  public:
   using core::Processor::Processor;
-  // Destructor
   ~BinFiles() override = default;
-  // Processor Name
-  static constexpr char const* ProcessorName = "BinFiles";
-  // Supported Properties
-  EXTENSIONAPI static core::Property MinSize;
-  EXTENSIONAPI static core::Property MaxSize;
-  EXTENSIONAPI static core::Property MinEntries;
-  EXTENSIONAPI static core::Property MaxEntries;
-  EXTENSIONAPI static core::Property MaxBinCount;
-  EXTENSIONAPI static core::Property MaxBinAge;
-  EXTENSIONAPI static core::Property BatchSize;
 
-  // Supported Relationships
-  EXTENSIONAPI static core::Relationship Failure;
-  EXTENSIONAPI static core::Relationship Original;
+  EXTENSIONAPI static constexpr const char* Description = "Bins flow files into buckets based on the number of entries or size of entries";
+
+  EXTENSIONAPI static const core::Property MinSize;
+  EXTENSIONAPI static const core::Property MaxSize;
+  EXTENSIONAPI static const core::Property MinEntries;
+  EXTENSIONAPI static const core::Property MaxEntries;
+  EXTENSIONAPI static const core::Property MaxBinCount;
+  EXTENSIONAPI static const core::Property MaxBinAge;
+  EXTENSIONAPI static const core::Property BatchSize;
+  static auto properties() {
+    return std::array{
+      MinSize,
+      MaxSize,
+      MinEntries,
+      MaxEntries,
+      MaxBinCount,
+      MaxBinAge,
+      BatchSize
+    };
+  }
+
+  EXTENSIONAPI static const core::Relationship Failure;
+  EXTENSIONAPI static const core::Relationship Original;
+  static auto relationships() {
+    return std::array{
+      Failure,
+      Original
+    };
+  }
+
+  EXTENSIONAPI static constexpr bool SupportsDynamicProperties = false;
+  EXTENSIONAPI static constexpr bool SupportsDynamicRelationships = false;
+  EXTENSIONAPI static constexpr core::annotation::Input InputRequirement = core::annotation::Input::INPUT_ALLOWED;
+  EXTENSIONAPI static constexpr bool IsSingleThreaded = false;
+
+  ADD_COMMON_VIRTUAL_FUNCTIONS_FOR_PROCESSORS
 
   // attributes
   EXTENSIONAPI static const char *FRAGMENT_ID_ATTRIBUTE;
@@ -227,19 +241,10 @@ class BinFiles : public core::Processor {
   EXTENSIONAPI static const char *TAR_PERMISSIONS_ATTRIBUTE;
 
  public:
-  /**
-   * Function that's executed when the processor is scheduled.
-   * @param context process context.
-   * @param sessionFactory process session factory that is used when creating
-   * ProcessSession objects.
-   */
   void onSchedule(core::ProcessContext *context, core::ProcessSessionFactory *sessionFactory) override;
-  // OnTrigger method, implemented by NiFi BinFiles
   void onTrigger(core::ProcessContext* /*context*/, core::ProcessSession* /*session*/) override {
   }
-  // OnTrigger method, implemented by NiFi BinFiles
   void onTrigger(const std::shared_ptr<core::ProcessContext> &context, const std::shared_ptr<core::ProcessSession> &session) override;
-  // Initialize, over write by NiFi BinFiles
   void initialize() override;
 
   void restore(const std::shared_ptr<core::FlowFile>& flowFile) override;
@@ -271,9 +276,5 @@ class BinFiles : public core::Processor {
   core::FlowFileStore file_store_;
 };
 
-} /* namespace processors */
-} /* namespace minifi */
-} /* namespace nifi */
-} /* namespace apache */
-} /* namespace org */
+}  // namespace org::apache::nifi::minifi::processors
 
diff --git a/extensions/libarchive/BinFilesStaticDefinitions.cpp b/extensions/libarchive/BinFilesStaticDefinitions.cpp
new file mode 100644
index 000000000..45a636184
--- /dev/null
+++ b/extensions/libarchive/BinFilesStaticDefinitions.cpp
@@ -0,0 +1,107 @@
+/**
+ * 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 "BinFiles.h"
+#include "MergeContent.h"
+#include "core/PropertyBuilder.h"
+#include "core/Resource.h"
+
+// FIXME(fgerlits): we need to put all these static definitions in a single file so that they are executed in this order at runtime
+// when https://issues.apache.org/jira/browse/MINIFICPP-1825 is closed, these definitions should be moved back to the cpp file of the class to which they belong
+
+namespace org::apache::nifi::minifi::processors {
+
+// BinFiles
+
+const core::Property BinFiles::MinSize(
+    core::PropertyBuilder::createProperty("Minimum Group Size")
+    ->withDescription("The minimum size of for the bundle")
+    ->withDefaultValue<uint64_t>(0)->build());
+const core::Property BinFiles::MaxSize(
+    core::PropertyBuilder::createProperty("Maximum Group Size")
+    ->withDescription("The maximum size for the bundle. If not specified, there is no maximum.")
+    ->withType(core::StandardValidators::get().UNSIGNED_LONG_VALIDATOR)->build());
+const core::Property BinFiles::MinEntries(
+    core::PropertyBuilder::createProperty("Minimum Number of Entries")
+    ->withDescription("The minimum number of files to include in a bundle")
+    ->withDefaultValue<uint32_t>(1)->build());
+const core::Property BinFiles::MaxEntries(
+    core::PropertyBuilder::createProperty("Maximum Number of Entries")
+    ->withDescription("The maximum number of files to include in a bundle. If not specified, there is no maximum.")
+    ->withType(core::StandardValidators::get().UNSIGNED_INT_VALIDATOR)->build());
+const core::Property BinFiles::MaxBinAge(
+    core::PropertyBuilder::createProperty("Max Bin Age")
+    ->withDescription("The maximum age of a Bin that will trigger a Bin to be complete. Expected format is <duration> <time unit>")
+    ->withType(core::StandardValidators::get().TIME_PERIOD_VALIDATOR)->build());
+const core::Property BinFiles::MaxBinCount(
+    core::PropertyBuilder::createProperty("Maximum number of Bins")
+    ->withDescription("Specifies the maximum number of bins that can be held in memory at any one time")
+    ->withDefaultValue<uint32_t>(100)->build());
+const core::Property BinFiles::BatchSize(
+    core::PropertyBuilder::createProperty("Batch Size")
+    ->withDescription("Maximum number of FlowFiles processed in a single session")
+    ->withDefaultValue<uint32_t>(1)->build());
+
+const core::Relationship BinFiles::Original("original", "The FlowFiles that were used to create the bundle");
+const core::Relationship BinFiles::Failure("failure", "If the bundle cannot be created, all FlowFiles that would have been used to create the bundle will be transferred to failure");
+
+REGISTER_RESOURCE(BinFiles, Processor);
+
+
+// MergeContent
+
+const core::Property MergeContent::MergeStrategy(
+  core::PropertyBuilder::createProperty("Merge Strategy")
+  ->withDescription("Defragment or Bin-Packing Algorithm")
+  ->withAllowableValues<std::string>({merge_content_options::MERGE_STRATEGY_DEFRAGMENT, merge_content_options::MERGE_STRATEGY_BIN_PACK})
+  ->withDefaultValue(merge_content_options::MERGE_STRATEGY_DEFRAGMENT)->build());
+const core::Property MergeContent::MergeFormat(
+  core::PropertyBuilder::createProperty("Merge Format")
+  ->withDescription("Merge Format")
+  ->withAllowableValues<std::string>({
+      merge_content_options::MERGE_FORMAT_CONCAT_VALUE,
+      merge_content_options::MERGE_FORMAT_TAR_VALUE,
+      merge_content_options::MERGE_FORMAT_ZIP_VALUE,
+      merge_content_options::MERGE_FORMAT_FLOWFILE_STREAM_V3_VALUE})
+  ->withDefaultValue(merge_content_options::MERGE_FORMAT_CONCAT_VALUE)->build());
+const core::Property MergeContent::CorrelationAttributeName("Correlation Attribute Name", "Correlation Attribute Name", "");
+const core::Property MergeContent::DelimiterStrategy(
+  core::PropertyBuilder::createProperty("Delimiter Strategy")
+  ->withDescription("Determines if Header, Footer, and Demarcator should point to files")
+  ->withAllowableValues<std::string>({merge_content_options::DELIMITER_STRATEGY_FILENAME, merge_content_options::DELIMITER_STRATEGY_TEXT})
+  ->withDefaultValue(merge_content_options::DELIMITER_STRATEGY_FILENAME)->build());
+const core::Property MergeContent::Header("Header File", "Filename specifying the header to use", "");
+const core::Property MergeContent::Footer("Footer File", "Filename specifying the footer to use", "");
+const core::Property MergeContent::Demarcator("Demarcator File", "Filename specifying the demarcator to use", "");
+const core::Property MergeContent::KeepPath(
+  core::PropertyBuilder::createProperty("Keep Path")
+  ->withDescription("If using the Zip or Tar Merge Format, specifies whether or not the FlowFiles' paths should be included in their entry")
+  ->withDefaultValue(false)->build());
+const core::Property MergeContent::AttributeStrategy(
+  core::PropertyBuilder::createProperty("Attribute Strategy")
+  ->withDescription("Determines which FlowFile attributes should be added to the bundle. If 'Keep All Unique Attributes' is selected, "
+                    "any attribute on any FlowFile that gets bundled will be kept unless its value conflicts with the value from another FlowFile "
+                    "(in which case neither, or none, of the conflicting attributes will be kept). If 'Keep Only Common Attributes' is selected, "
+                    "only the attributes that exist on all FlowFiles in the bundle, with the same value, will be preserved.")
+  ->withAllowableValues<std::string>({merge_content_options::ATTRIBUTE_STRATEGY_KEEP_COMMON, merge_content_options::ATTRIBUTE_STRATEGY_KEEP_ALL_UNIQUE})
+  ->withDefaultValue(merge_content_options::ATTRIBUTE_STRATEGY_KEEP_COMMON)->build());
+
+const core::Relationship MergeContent::Merge("merged", "The FlowFile containing the merged content");
+
+REGISTER_RESOURCE(MergeContent, Processor);
+
+}  // namespace org::apache::nifi::minifi::processors
diff --git a/extensions/libarchive/CompressContent.cpp b/extensions/libarchive/CompressContent.cpp
index c03e63809..d2d6c484d 100644
--- a/extensions/libarchive/CompressContent.cpp
+++ b/extensions/libarchive/CompressContent.cpp
@@ -22,9 +22,9 @@
 #include <memory>
 #include <string>
 #include <map>
-#include <set>
 #include "core/ProcessContext.h"
 #include "core/ProcessSession.h"
+#include "core/PropertyBuilder.h"
 #include "utils/StringUtils.h"
 #include "core/Resource.h"
 #include "io/StreamPipe.h"
@@ -35,35 +35,35 @@ namespace nifi {
 namespace minifi {
 namespace processors {
 
-core::Property CompressContent::CompressLevel(
+const core::Property CompressContent::CompressLevel(
     core::PropertyBuilder::createProperty("Compression Level")->withDescription("The compression level to use; this is valid only when using GZIP compression.")
         ->isRequired(false)->withDefaultValue<int>(1)->build());
-core::Property CompressContent::CompressMode(
+const core::Property CompressContent::CompressMode(
     core::PropertyBuilder::createProperty("Mode")->withDescription("Indicates whether the processor should compress content or decompress content.")
         ->isRequired(false)->withAllowableValues(CompressionMode::values())
         ->withDefaultValue(toString(CompressionMode::Compress))->build());
-core::Property CompressContent::CompressFormat(
+const core::Property CompressContent::CompressFormat(
     core::PropertyBuilder::createProperty("Compression Format")->withDescription("The compression format to use.")
         ->isRequired(false)
         ->withAllowableValues(ExtendedCompressionFormat::values())
         ->withDefaultValue(toString(ExtendedCompressionFormat::USE_MIME_TYPE))->build());
-core::Property CompressContent::UpdateFileName(
+const core::Property CompressContent::UpdateFileName(
     core::PropertyBuilder::createProperty("Update Filename")->withDescription("Determines if filename extension need to be updated")
         ->isRequired(false)->withDefaultValue<bool>(false)->build());
-core::Property CompressContent::EncapsulateInTar(
+const core::Property CompressContent::EncapsulateInTar(
     core::PropertyBuilder::createProperty("Encapsulate in TAR")
         ->withDescription("If true, on compression the FlowFile is added to a TAR archive and then compressed, "
                           "and on decompression a compressed, TAR-encapsulated FlowFile is expected.\n"
                           "If false, on compression the content of the FlowFile simply gets compressed, and on decompression a simple compressed content is expected.\n"
                           "true is the behaviour compatible with older MiNiFi C++ versions, false is the behaviour compatible with NiFi.")
         ->isRequired(false)->withDefaultValue<bool>(true)->build());
-core::Property CompressContent::BatchSize(
+const core::Property CompressContent::BatchSize(
     core::PropertyBuilder::createProperty("Batch Size")
     ->withDescription("Maximum number of FlowFiles processed in a single session")
     ->withDefaultValue<uint32_t>(1)->build());
 
-core::Relationship CompressContent::Success("success", "FlowFiles will be transferred to the success relationship after successfully being compressed or decompressed");
-core::Relationship CompressContent::Failure("failure", "FlowFiles will be transferred to the failure relationship if they fail to compress/decompress");
+const core::Relationship CompressContent::Success("success", "FlowFiles will be transferred to the success relationship after successfully being compressed or decompressed");
+const core::Relationship CompressContent::Failure("failure", "FlowFiles will be transferred to the failure relationship if they fail to compress/decompress");
 
 const std::string CompressContent::TAR_EXT = ".tar";
 
@@ -83,20 +83,8 @@ const std::map<io::CompressionFormat, std::string> CompressContent::fileExtensio
 };
 
 void CompressContent::initialize() {
-  // Set the supported properties
-  std::set<core::Property> properties;
-  properties.insert(CompressLevel);
-  properties.insert(CompressMode);
-  properties.insert(CompressFormat);
-  properties.insert(UpdateFileName);
-  properties.insert(EncapsulateInTar);
-  properties.insert(BatchSize);
-  setSupportedProperties(properties);
-  // Set the supported relationships
-  std::set<core::Relationship> relationships;
-  relationships.insert(Failure);
-  relationships.insert(Success);
-  setSupportedRelationships(relationships);
+  setSupportedProperties(properties());
+  setSupportedRelationships(relationships());
 }
 
 void CompressContent::onSchedule(core::ProcessContext *context, core::ProcessSessionFactory* /*sessionFactory*/) {
@@ -258,7 +246,7 @@ std::string CompressContent::toMimeType(io::CompressionFormat format) {
   throw Exception(GENERAL_EXCEPTION, "Invalid compression format");
 }
 
-REGISTER_RESOURCE(CompressContent, "Compresses or decompresses the contents of FlowFiles using a user-specified compression algorithm and updates the mime.type attribute as appropriate");
+REGISTER_RESOURCE(CompressContent, Processor);
 
 } /* namespace processors */
 } /* namespace minifi */
diff --git a/extensions/libarchive/CompressContent.h b/extensions/libarchive/CompressContent.h
index f41adba00..59edb2da5 100644
--- a/extensions/libarchive/CompressContent.h
+++ b/extensions/libarchive/CompressContent.h
@@ -48,33 +48,45 @@ namespace nifi {
 namespace minifi {
 namespace processors {
 
-// CompressContent Class
 class CompressContent : public core::Processor {
  public:
-  // Constructor
-  /*!
-   * Create a new processor
-   */
   explicit CompressContent(const std::string& name, const utils::Identifier& uuid = {})
     : core::Processor(name, uuid)
     , updateFileName_(false)
     , encapsulateInTar_(false) {
   }
-  // Destructor
   ~CompressContent() override = default;
-  // Processor Name
-  EXTENSIONAPI static constexpr char const* ProcessorName = "CompressContent";
-  // Supported Properties
-  EXTENSIONAPI static core::Property CompressMode;
-  EXTENSIONAPI static core::Property CompressLevel;
-  EXTENSIONAPI static core::Property CompressFormat;
-  EXTENSIONAPI static core::Property UpdateFileName;
-  EXTENSIONAPI static core::Property EncapsulateInTar;
-  EXTENSIONAPI static core::Property BatchSize;
-
-  // Supported Relationships
-  EXTENSIONAPI static core::Relationship Failure;
-  EXTENSIONAPI static core::Relationship Success;
+
+  EXTENSIONAPI static constexpr const char* Description = "Compresses or decompresses the contents of FlowFiles using a user-specified compression algorithm "
+      "and updates the mime.type attribute as appropriate";
+
+  EXTENSIONAPI static const core::Property CompressMode;
+  EXTENSIONAPI static const core::Property CompressLevel;
+  EXTENSIONAPI static const core::Property CompressFormat;
+  EXTENSIONAPI static const core::Property UpdateFileName;
+  EXTENSIONAPI static const core::Property EncapsulateInTar;
+  EXTENSIONAPI static const core::Property BatchSize;
+  static auto properties() {
+    return std::array{
+      CompressMode,
+      CompressLevel,
+      CompressFormat,
+      UpdateFileName,
+      EncapsulateInTar,
+      BatchSize
+    };
+  }
+
+  EXTENSIONAPI static const core::Relationship Success;
+  EXTENSIONAPI static const core::Relationship Failure;
+  static auto relationships() { return std::array{Success, Failure}; }
+
+  EXTENSIONAPI static constexpr bool SupportsDynamicProperties = false;
+  EXTENSIONAPI static constexpr bool SupportsDynamicRelationships = false;
+  EXTENSIONAPI static constexpr core::annotation::Input InputRequirement = core::annotation::Input::INPUT_REQUIRED;
+  EXTENSIONAPI static constexpr bool IsSingleThreaded = false;
+
+  ADD_COMMON_VIRTUAL_FUNCTIONS_FOR_PROCESSORS
 
   static const std::string TAR_EXT;
 
@@ -159,10 +171,6 @@ class CompressContent : public core::Processor {
 
   void processFlowFile(const std::shared_ptr<core::FlowFile>& flowFile, const std::shared_ptr<core::ProcessSession>& session);
 
-  core::annotation::Input getInputRequirement() const override {
-    return core::annotation::Input::INPUT_REQUIRED;
-  }
-
   std::shared_ptr<core::logging::Logger> logger_ = core::logging::LoggerFactory<CompressContent>::getLogger();
   int compressLevel_{};
   CompressionMode compressMode_;
diff --git a/extensions/libarchive/FocusArchiveEntry.cpp b/extensions/libarchive/FocusArchiveEntry.cpp
index ad5c1b282..5cf1e93ed 100644
--- a/extensions/libarchive/FocusArchiveEntry.cpp
+++ b/extensions/libarchive/FocusArchiveEntry.cpp
@@ -26,7 +26,6 @@
 
 #include <array>
 #include <string>
-#include <set>
 
 #include <memory>
 
@@ -44,18 +43,12 @@ namespace processors {
 
 std::shared_ptr<utils::IdGenerator> FocusArchiveEntry::id_generator_ = utils::IdGenerator::getIdGenerator();
 
-core::Property FocusArchiveEntry::Path("Path", "The path within the archive to focus (\"/\" to focus the total archive)", "");
-core::Relationship FocusArchiveEntry::Success("success", "success operational on the flow record");
+const core::Property FocusArchiveEntry::Path("Path", "The path within the archive to focus (\"/\" to focus the total archive)", "");
+const core::Relationship FocusArchiveEntry::Success("success", "success operational on the flow record");
 
 void FocusArchiveEntry::initialize() {
-  //! Set the supported properties
-  std::set<core::Property> properties;
-  properties.insert(Path);
-  setSupportedProperties(properties);
-  //! Set the supported relationships
-  std::set<core::Relationship> relationships;
-  relationships.insert(Success);
-  setSupportedRelationships(relationships);
+  setSupportedProperties(properties());
+  setSupportedRelationships(relationships());
 }
 
 void FocusArchiveEntry::onTrigger(core::ProcessContext *context, core::ProcessSession *session) {
@@ -252,9 +245,7 @@ FocusArchiveEntry::ReadCallback::ReadCallback(core::Processor *processor, utils:
   _archiveMetadata = archiveMetadata;
 }
 
-REGISTER_RESOURCE(FocusArchiveEntry, "Allows manipulation of entries within an archive (e.g. TAR) by focusing on one entry within the archive at a time. "
-    "When an archive entry is focused, that entry is treated as the content of the FlowFile and may be manipulated independently of the rest of the archive."
-    " To restore the FlowFile to its original state, use UnfocusArchiveEntry.");
+REGISTER_RESOURCE(FocusArchiveEntry, Processor);
 
 } /* namespace processors */
 } /* namespace minifi */
diff --git a/extensions/libarchive/FocusArchiveEntry.h b/extensions/libarchive/FocusArchiveEntry.h
index 9b5d3e16e..15ca4ff53 100644
--- a/extensions/libarchive/FocusArchiveEntry.h
+++ b/extensions/libarchive/FocusArchiveEntry.h
@@ -1,7 +1,4 @@
 /**
- * @file FocusArchiveEntry.h
- * FocusArchiveEntry class declaration
- *
  * 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.
@@ -39,29 +36,31 @@ namespace nifi {
 namespace minifi {
 namespace processors {
 
-//! FocusArchiveEntry Class
 class FocusArchiveEntry : public core::Processor {
  public:
-  //! Constructor
-  /*!
-   * Create a new processor
-   */
   explicit FocusArchiveEntry(const std::string& name, const utils::Identifier& uuid = {})
   : core::Processor(name, uuid) {
   }
-  //! Destructor
   ~FocusArchiveEntry()   override = default;
-  //! Processor Name
-  EXTENSIONAPI static constexpr char const* ProcessorName = "FocusArchiveEntry";
-  //! Supported Properties
-  EXTENSIONAPI static core::Property Path;
-  //! Supported Relationships
-  EXTENSIONAPI static core::Relationship Success;
 
-  //! OnTrigger method, implemented by NiFi FocusArchiveEntry
-  void onTrigger(core::ProcessContext *context,
-      core::ProcessSession *session) override;
-  //! Initialize, over write by NiFi FocusArchiveEntry
+  EXTENSIONAPI static constexpr const char* Description = "Allows manipulation of entries within an archive (e.g. TAR) by focusing on one entry within the archive at a time. "
+      "When an archive entry is focused, that entry is treated as the content of the FlowFile and may be manipulated independently of the rest of the archive. "
+      "To restore the FlowFile to its original state, use UnfocusArchiveEntry.";
+
+  EXTENSIONAPI static const core::Property Path;
+  static auto properties() { return std::array{Path}; }
+
+  EXTENSIONAPI static const core::Relationship Success;
+  static auto relationships() { return std::array{Success}; }
+
+  EXTENSIONAPI static constexpr bool SupportsDynamicProperties = false;
+  EXTENSIONAPI static constexpr bool SupportsDynamicRelationships = false;
+  EXTENSIONAPI static constexpr core::annotation::Input InputRequirement = core::annotation::Input::INPUT_ALLOWED;
+  EXTENSIONAPI static constexpr bool IsSingleThreaded = false;
+
+  ADD_COMMON_VIRTUAL_FUNCTIONS_FOR_PROCESSORS
+
+  void onTrigger(core::ProcessContext *context, core::ProcessSession *session) override;
   void initialize() override;
 
   class ReadCallback {
@@ -80,7 +79,6 @@ class FocusArchiveEntry : public core::Processor {
   };
 
  private:
-  //! Logger
   std::shared_ptr<core::logging::Logger> logger_ = core::logging::LoggerFactory<FocusArchiveEntry>::getLogger();
   static std::shared_ptr<utils::IdGenerator> id_generator_;
 };
diff --git a/extensions/libarchive/ManipulateArchive.cpp b/extensions/libarchive/ManipulateArchive.cpp
index b62dcbab2..50e05d7e3 100644
--- a/extensions/libarchive/ManipulateArchive.cpp
+++ b/extensions/libarchive/ManipulateArchive.cpp
@@ -20,7 +20,6 @@
 #include <iostream>
 #include <memory>
 #include <string>
-#include <set>
 #include <list>
 #include <algorithm>
 
@@ -43,13 +42,14 @@ namespace nifi {
 namespace minifi {
 namespace processors {
 
-core::Property ManipulateArchive::Operation("Operation", "Operation to perform on the archive (touch, remove, copy, move).", "");
-core::Property ManipulateArchive::Target("Target", "An existing entry within the archive to perform the operation on.", "");
-core::Property ManipulateArchive::Destination("Destination", "Destination for operations (touch, move or copy) which result in new entries.", "");
-core::Property ManipulateArchive::Before("Before", "For operations which result in new entries, places the new entry before the entry specified by this property.", "");
-core::Property ManipulateArchive::After("After", "For operations which result in new entries, places the new entry after the entry specified by this property.", "");
-core::Relationship ManipulateArchive::Success("success", "FlowFiles will be transferred to the success relationship if the operation succeeds.");
-core::Relationship ManipulateArchive::Failure("failure", "FlowFiles will be transferred to the failure relationship if the operation fails.");
+const core::Property ManipulateArchive::Operation("Operation", "Operation to perform on the archive (touch, remove, copy, move).", "");
+const core::Property ManipulateArchive::Target("Target", "An existing entry within the archive to perform the operation on.", "");
+const core::Property ManipulateArchive::Destination("Destination", "Destination for operations (touch, move or copy) which result in new entries.", "");
+const core::Property ManipulateArchive::Before("Before", "For operations which result in new entries, places the new entry before the entry specified by this property.", "");
+const core::Property ManipulateArchive::After("After", "For operations which result in new entries, places the new entry after the entry specified by this property.", "");
+
+const core::Relationship ManipulateArchive::Success("success", "FlowFiles will be transferred to the success relationship if the operation succeeds.");
+const core::Relationship ManipulateArchive::Failure("failure", "FlowFiles will be transferred to the failure relationship if the operation fails.");
 
 char const* ManipulateArchive::OPERATION_REMOVE = "remove";
 char const* ManipulateArchive::OPERATION_COPY =   "copy";
@@ -57,20 +57,9 @@ char const* ManipulateArchive::OPERATION_MOVE =   "move";
 char const* ManipulateArchive::OPERATION_TOUCH =  "touch";
 
 void ManipulateArchive::initialize() {
-    //! Set the supported properties
-    std::set<core::Property> properties;
-    properties.insert(Operation);
-    properties.insert(Target);
-    properties.insert(Destination);
-    properties.insert(Before);
-    properties.insert(After);
-    setSupportedProperties(properties);
-
-    //! Set the supported relationships
-    std::set<core::Relationship> relationships;
-    relationships.insert(Success);
-    relationships.insert(Failure);
-    setSupportedRelationships(relationships);
+  setSupportedProperties(properties());
+
+  setSupportedRelationships(relationships());
 }
 
 void ManipulateArchive::onSchedule(core::ProcessContext *context, core::ProcessSessionFactory* /*sessionFactory*/) {
@@ -213,7 +202,7 @@ void ManipulateArchive::onTrigger(core::ProcessContext* /*context*/, core::Proce
     session->transfer(flowFile, Success);
 }
 
-REGISTER_RESOURCE(ManipulateArchive, "Performs an operation which manipulates an archive without needing to split the archive into multiple FlowFiles.");
+REGISTER_RESOURCE(ManipulateArchive, Processor);
 
 } /* namespace processors */
 } /* namespace minifi */
diff --git a/extensions/libarchive/ManipulateArchive.h b/extensions/libarchive/ManipulateArchive.h
index d71c54fd1..b52286fa3 100644
--- a/extensions/libarchive/ManipulateArchive.h
+++ b/extensions/libarchive/ManipulateArchive.h
@@ -43,7 +43,34 @@ class ManipulateArchive : public core::Processor {
       : core::Processor(name, uuid) {
   }
   ~ManipulateArchive() override = default;
-  EXTENSIONAPI static constexpr char const* ProcessorName = "ManipulateArchive";
+
+  EXTENSIONAPI static constexpr const char* Description = "Performs an operation which manipulates an archive without needing to split the archive into multiple FlowFiles.";
+
+  EXTENSIONAPI static const core::Property Operation;
+  EXTENSIONAPI static const core::Property Target;
+  EXTENSIONAPI static const core::Property Destination;
+  EXTENSIONAPI static const core::Property Before;
+  EXTENSIONAPI static const core::Property After;
+  static auto properties() {
+    return std::array{
+      Operation,
+      Target,
+      Destination,
+      Before,
+      After
+    };
+  }
+
+  EXTENSIONAPI static const core::Relationship Success;
+  EXTENSIONAPI static const core::Relationship Failure;
+  static auto relationships() { return std::array{Success, Failure}; }
+
+  EXTENSIONAPI static constexpr bool SupportsDynamicProperties = false;
+  EXTENSIONAPI static constexpr bool SupportsDynamicRelationships = false;
+  EXTENSIONAPI static constexpr core::annotation::Input InputRequirement = core::annotation::Input::INPUT_ALLOWED;
+  EXTENSIONAPI static constexpr bool IsSingleThreaded = false;
+
+  ADD_COMMON_VIRTUAL_FUNCTIONS_FOR_PROCESSORS
 
   // Supported operations
   EXTENSIONAPI static char const* OPERATION_REMOVE;
@@ -51,24 +78,11 @@ class ManipulateArchive : public core::Processor {
   EXTENSIONAPI static char const* OPERATION_MOVE;
   EXTENSIONAPI static char const* OPERATION_TOUCH;
 
-  // Supported Properties
-  EXTENSIONAPI static core::Property Operation;
-  EXTENSIONAPI static core::Property Target;
-  EXTENSIONAPI static core::Property Destination;
-  EXTENSIONAPI static core::Property Before;
-  EXTENSIONAPI static core::Property After;
-  // Supported Relationships
-  EXTENSIONAPI static core::Relationship Success;
-  EXTENSIONAPI static core::Relationship Failure;
-
-  // OnTrigger method, implemented by NiFi ManipulateArchive
   void onTrigger(core::ProcessContext *context, core::ProcessSession *session) override;
   void onSchedule(core::ProcessContext *context, core::ProcessSessionFactory *sessionFactory) override;
-  // Initialize, over write by NiFi ManipulateArchive
   void initialize() override;
 
  private:
-  // Logger
   std::shared_ptr<Logger> logger_ = core::logging::LoggerFactory<ManipulateArchive>::getLogger();
   std::string before_, after_, operation_, destination_, targetEntry_;
 };
diff --git a/extensions/libarchive/MergeContent.cpp b/extensions/libarchive/MergeContent.cpp
index d0f4461ab..bf17eac5a 100644
--- a/extensions/libarchive/MergeContent.cpp
+++ b/extensions/libarchive/MergeContent.cpp
@@ -1,7 +1,4 @@
 /**
- * @file MergeContent.cpp
- * MergeContent class implementation
- *
  * 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.
@@ -21,88 +18,22 @@
 #include <stdio.h>
 #include <memory>
 #include <string>
-#include <set>
 #include <map>
 #include <deque>
 #include <utility>
 #include <algorithm>
 #include <numeric>
-#include "utils/TimeUtil.h"
 #include "core/ProcessContext.h"
 #include "core/ProcessSession.h"
 #include "core/Resource.h"
 #include "serialization/PayloadSerializer.h"
 #include "serialization/FlowFileV3Serializer.h"
 
-namespace org {
-namespace apache {
-namespace nifi {
-namespace minifi {
-namespace processors {
-
-core::Property MergeContent::MergeStrategy(
-  core::PropertyBuilder::createProperty("Merge Strategy")
-  ->withDescription("Defragment or Bin-Packing Algorithm")
-  ->withAllowableValues<std::string>({merge_content_options::MERGE_STRATEGY_DEFRAGMENT, merge_content_options::MERGE_STRATEGY_BIN_PACK})
-  ->withDefaultValue(merge_content_options::MERGE_STRATEGY_DEFRAGMENT)->build());
-core::Property MergeContent::MergeFormat(
-  core::PropertyBuilder::createProperty("Merge Format")
-  ->withDescription("Merge Format")
-  ->withAllowableValues<std::string>({
-      merge_content_options::MERGE_FORMAT_CONCAT_VALUE,
-      merge_content_options::MERGE_FORMAT_TAR_VALUE,
-      merge_content_options::MERGE_FORMAT_ZIP_VALUE,
-      merge_content_options::MERGE_FORMAT_FLOWFILE_STREAM_V3_VALUE})
-  ->withDefaultValue(merge_content_options::MERGE_FORMAT_CONCAT_VALUE)->build());
-core::Property MergeContent::CorrelationAttributeName("Correlation Attribute Name", "Correlation Attribute Name", "");
-core::Property MergeContent::DelimiterStrategy(
-  core::PropertyBuilder::createProperty("Delimiter Strategy")
-  ->withDescription("Determines if Header, Footer, and Demarcator should point to files")
-  ->withAllowableValues<std::string>({merge_content_options::DELIMITER_STRATEGY_FILENAME, merge_content_options::DELIMITER_STRATEGY_TEXT})
-  ->withDefaultValue(merge_content_options::DELIMITER_STRATEGY_FILENAME)->build());
-core::Property MergeContent::Header("Header File", "Filename specifying the header to use", "");
-core::Property MergeContent::Footer("Footer File", "Filename specifying the footer to use", "");
-core::Property MergeContent::Demarcator("Demarcator File", "Filename specifying the demarcator to use", "");
-core::Property MergeContent::KeepPath(
-  core::PropertyBuilder::createProperty("Keep Path")
-  ->withDescription("If using the Zip or Tar Merge Format, specifies whether or not the FlowFiles' paths should be included in their entry")
-  ->withDefaultValue(false)->build());
-core::Property MergeContent::AttributeStrategy(
-  core::PropertyBuilder::createProperty("Attribute Strategy")
-  ->withDescription("Determines which FlowFile attributes should be added to the bundle. If 'Keep All Unique Attributes' is selected, "
-                    "any attribute on any FlowFile that gets bundled will be kept unless its value conflicts with the value from another FlowFile "
-                    "(in which case neither, or none, of the conflicting attributes will be kept). If 'Keep Only Common Attributes' is selected, "
-                    "only the attributes that exist on all FlowFiles in the bundle, with the same value, will be preserved.")
-  ->withAllowableValues<std::string>({merge_content_options::ATTRIBUTE_STRATEGY_KEEP_COMMON, merge_content_options::ATTRIBUTE_STRATEGY_KEEP_ALL_UNIQUE})
-  ->withDefaultValue(merge_content_options::ATTRIBUTE_STRATEGY_KEEP_COMMON)->build());
-core::Relationship MergeContent::Merge("merged", "The FlowFile containing the merged content");
+namespace org::apache::nifi::minifi::processors {
 
 void MergeContent::initialize() {
-  // Set the supported properties
-  std::set<core::Property> properties;
-  properties.insert(MinSize);
-  properties.insert(MaxSize);
-  properties.insert(MinEntries);
-  properties.insert(MaxEntries);
-  properties.insert(MaxBinAge);
-  properties.insert(MaxBinCount);
-  properties.insert(BatchSize);
-  properties.insert(MergeStrategy);
-  properties.insert(MergeFormat);
-  properties.insert(CorrelationAttributeName);
-  properties.insert(DelimiterStrategy);
-  properties.insert(Header);
-  properties.insert(Footer);
-  properties.insert(Demarcator);
-  properties.insert(KeepPath);
-  properties.insert(AttributeStrategy);
-  setSupportedProperties(properties);
-  // Set the supported relationships
-  std::set<core::Relationship> relationships;
-  relationships.insert(Original);
-  relationships.insert(Failure);
-  relationships.insert(Merge);
-  setSupportedRelationships(relationships);
+  setSupportedProperties(properties());
+  setSupportedRelationships(relationships());
 }
 
 std::string MergeContent::readContent(std::string path) {
@@ -432,12 +363,4 @@ void KeepAllUniqueAttributesMerger::processFlowFile(const std::shared_ptr<core::
   }
 }
 
-REGISTER_RESOURCE(MergeContent, "Merges a Group of FlowFiles together based on a user-defined strategy and packages them into a single FlowFile. "
-    "MergeContent should be configured with only one incoming connection as it won't create grouped Flow Files."
-    "This processor updates the mime.type attribute as appropriate.");
-
-} /* namespace processors */
-} /* namespace minifi */
-} /* namespace nifi */
-} /* namespace apache */
-} /* namespace org */
+}  // namespace org::apache::nifi::minifi::processors
diff --git a/extensions/libarchive/MergeContent.h b/extensions/libarchive/MergeContent.h
index 7cb6b24fe..66ff747f3 100644
--- a/extensions/libarchive/MergeContent.h
+++ b/extensions/libarchive/MergeContent.h
@@ -1,7 +1,4 @@
 /**
- * @file MergeContent.h
- * MergeContent class declaration
- *
  * 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.
@@ -32,14 +29,11 @@
 #include "archive.h"
 #include "core/logging/LoggerConfiguration.h"
 #include "serialization/FlowFileSerializer.h"
+#include "utils/ArrayUtils.h"
 #include "utils/gsl.h"
 #include "utils/Export.h"
 
-namespace org {
-namespace apache {
-namespace nifi {
-namespace minifi {
-namespace processors {
+namespace org::apache::nifi::minifi::processors {
 
 namespace merge_content_options {
 
@@ -56,7 +50,6 @@ constexpr const char *ATTRIBUTE_STRATEGY_KEEP_ALL_UNIQUE = "Keep All Unique Attr
 
 } /* namespace merge_content_options */
 
-// MergeBin Class
 class MergeBin {
  public:
   virtual ~MergeBin() = default;
@@ -65,7 +58,6 @@ class MergeBin {
       std::deque<std::shared_ptr<core::FlowFile>> &flows, FlowFileSerializer& serializer, const std::shared_ptr<core::FlowFile> &flowFile) = 0;
 };
 
-// BinaryConcatenationMerge Class
 class BinaryConcatenationMerge : public MergeBin {
  public:
   BinaryConcatenationMerge(const std::string& header, const std::string& footer, const std::string& demarcator);
@@ -125,7 +117,6 @@ class BinaryConcatenationMerge : public MergeBin {
 };
 
 
-// Archive Class
 class ArchiveMerge {
  public:
   class ArchiveWriter : public io::OutputStream {
@@ -247,14 +238,12 @@ class ArchiveMerge {
   };
 };
 
-// TarMerge Class
 class TarMerge: public ArchiveMerge, public MergeBin {
  public:
   void merge(core::ProcessContext *context, core::ProcessSession *session, std::deque<std::shared_ptr<core::FlowFile>> &flows,
              FlowFileSerializer& serializer, const std::shared_ptr<core::FlowFile> &merge_flow) override;
 };
 
-// ZipMerge Class
 class ZipMerge: public ArchiveMerge, public MergeBin {
  public:
   void merge(core::ProcessContext *context, core::ProcessSession *session, std::deque<std::shared_ptr<core::FlowFile>> &flows,
@@ -296,13 +285,8 @@ class KeepAllUniqueAttributesMerger: public AttributeMerger {
   std::vector<std::string> removed_attributes_;
 };
 
-// MergeContent Class
 class MergeContent : public processors::BinFiles {
  public:
-  // Constructor
-  /*!
-   * Create a new processor
-   */
   explicit MergeContent(const std::string& name, const utils::Identifier& uuid = {})
       : processors::BinFiles(name, uuid) {
     mergeStrategy_ = merge_content_options::MERGE_STRATEGY_DEFRAGMENT;
@@ -311,35 +295,49 @@ class MergeContent : public processors::BinFiles {
     keepPath_ = false;
     attributeStrategy_ = merge_content_options::ATTRIBUTE_STRATEGY_KEEP_COMMON;
   }
-  // Destructor
   ~MergeContent() override = default;
-  // Processor Name
-  EXTENSIONAPI static constexpr char const* ProcessorName = "MergeContent";
-  // Supported Properties
-  EXTENSIONAPI static core::Property MergeStrategy;
-  EXTENSIONAPI static core::Property MergeFormat;
-  EXTENSIONAPI static core::Property CorrelationAttributeName;
-  EXTENSIONAPI static core::Property DelimiterStrategy;
-  EXTENSIONAPI static core::Property KeepPath;
-  EXTENSIONAPI static core::Property Header;
-  EXTENSIONAPI static core::Property Footer;
-  EXTENSIONAPI static core::Property Demarcator;
-  EXTENSIONAPI static core::Property AttributeStrategy;
-
-  // Supported Relationships
-  EXTENSIONAPI static core::Relationship Merge;
 
- public:
-  /**
-   * Function that's executed when the processor is scheduled.
-   * @param context process context.
-   * @param sessionFactory process session factory that is used when creating
-   * ProcessSession objects.
-   */
+  EXTENSIONAPI static constexpr const char* Description = "Merges a Group of FlowFiles together based on a user-defined strategy and packages them into a single FlowFile. "
+      "MergeContent should be configured with only one incoming connection as it won't create grouped Flow Files."
+      "This processor updates the mime.type attribute as appropriate.";
+
+  EXTENSIONAPI static const core::Property MergeStrategy;
+  EXTENSIONAPI static const core::Property MergeFormat;
+  EXTENSIONAPI static const core::Property CorrelationAttributeName;
+  EXTENSIONAPI static const core::Property DelimiterStrategy;
+  EXTENSIONAPI static const core::Property KeepPath;
+  EXTENSIONAPI static const core::Property Header;
+  EXTENSIONAPI static const core::Property Footer;
+  EXTENSIONAPI static const core::Property Demarcator;
+  EXTENSIONAPI static const core::Property AttributeStrategy;
+  static auto properties() {
+    return utils::array_cat(BinFiles::properties(), std::array{
+      MergeStrategy,
+      MergeFormat,
+      CorrelationAttributeName,
+      DelimiterStrategy,
+      KeepPath,
+      Header,
+      Footer,
+      Demarcator,
+      AttributeStrategy
+    });
+  }
+
+  EXTENSIONAPI static const core::Relationship Merge;
+  static auto relationships() {
+    return utils::array_cat(BinFiles::relationships(), std::array{Merge});
+  }
+
+  EXTENSIONAPI static constexpr bool SupportsDynamicProperties = false;
+  EXTENSIONAPI static constexpr bool SupportsDynamicRelationships = false;
+  EXTENSIONAPI static constexpr core::annotation::Input InputRequirement = core::annotation::Input::INPUT_REQUIRED;
+  EXTENSIONAPI static constexpr bool IsSingleThreaded = false;
+
+  ADD_COMMON_VIRTUAL_FUNCTIONS_FOR_PROCESSORS
+
   void onSchedule(core::ProcessContext *context, core::ProcessSessionFactory *sessionFactory) override;
-  // OnTrigger method, implemented by NiFi MergeContent
   void onTrigger(core::ProcessContext *context, core::ProcessSession *session) override;
-  // Initialize, over write by NiFi MergeContent
   void initialize() override;
   bool processBin(core::ProcessContext *context, core::ProcessSession *session, std::unique_ptr<Bin> &bin) override;
 
@@ -352,10 +350,6 @@ class MergeContent : public processors::BinFiles {
  private:
   void validatePropertyOptions();
 
-  core::annotation::Input getInputRequirement() const override {
-    return core::annotation::Input::INPUT_REQUIRED;
-  }
-
   std::shared_ptr<core::logging::Logger> logger_ = core::logging::LoggerFactory<MergeContent>::getLogger();
   std::string mergeStrategy_;
   std::string mergeFormat_;
@@ -369,12 +363,7 @@ class MergeContent : public processors::BinFiles {
   std::string footerContent_;
   std::string demarcatorContent_;
   std::string attributeStrategy_;
-  // readContent
   std::string readContent(std::string path);
 };
 
-} /* namespace processors */
-} /* namespace minifi */
-} /* namespace nifi */
-} /* namespace apache */
-} /* namespace org */
+}  // namespace org::apache::nifi::minifi::processors
diff --git a/extensions/libarchive/UnfocusArchiveEntry.cpp b/extensions/libarchive/UnfocusArchiveEntry.cpp
index fc1f94b94..43d0a4cde 100644
--- a/extensions/libarchive/UnfocusArchiveEntry.cpp
+++ b/extensions/libarchive/UnfocusArchiveEntry.cpp
@@ -24,7 +24,6 @@
 #include <fstream>
 #include <memory>
 #include <string>
-#include <set>
 #include <system_error>
 
 #include "archive.h"
@@ -41,16 +40,11 @@ namespace nifi {
 namespace minifi {
 namespace processors {
 
-core::Relationship UnfocusArchiveEntry::Success("success", "success operational on the flow record");
+const core::Relationship UnfocusArchiveEntry::Success("success", "success operational on the flow record");
 
 void UnfocusArchiveEntry::initialize() {
-  //! Set the supported properties
-  std::set<core::Property> properties;
-  setSupportedProperties(properties);
-  //! Set the supported relationships
-  std::set<core::Relationship> relationships;
-  relationships.insert(Success);
-  setSupportedRelationships(relationships);
+  setSupportedProperties(properties());
+  setSupportedRelationships(relationships());
 }
 
 void UnfocusArchiveEntry::onTrigger(core::ProcessContext *context, core::ProcessSession *session) {
@@ -235,7 +229,7 @@ int64_t UnfocusArchiveEntry::WriteCallback::operator()(const std::shared_ptr<io:
   return nlen;
 }
 
-REGISTER_RESOURCE(UnfocusArchiveEntry, "Restores a FlowFile which has had an archive entry focused via FocusArchiveEntry to its original state.");
+REGISTER_RESOURCE(UnfocusArchiveEntry, Processor);
 
 } /* namespace processors */
 } /* namespace minifi */
diff --git a/extensions/libarchive/UnfocusArchiveEntry.h b/extensions/libarchive/UnfocusArchiveEntry.h
index 905e886f5..57f699aa7 100644
--- a/extensions/libarchive/UnfocusArchiveEntry.h
+++ b/extensions/libarchive/UnfocusArchiveEntry.h
@@ -40,27 +40,28 @@ namespace processors {
 
 using core::logging::Logger;
 
-//! UnfocusArchiveEntry Class
 class UnfocusArchiveEntry : public core::Processor {
  public:
-  //! Constructor
-  /*!
-   * Create a new processor
-   */
   explicit UnfocusArchiveEntry(const std::string& name, const utils::Identifier& uuid = {})
       : core::Processor(name, uuid) {
   }
-  //! Destructor
   ~UnfocusArchiveEntry() override = default;
-  //! Processor Name
-  static constexpr char const* ProcessorName = "UnfocusArchiveEntry";
-  //! Supported Relationships
-  static core::Relationship Success;
 
-  //! OnTrigger method, implemented by NiFi UnfocusArchiveEntry
-  void onTrigger(core::ProcessContext *context,
-      core::ProcessSession *session) override;
-  //! Initialize, over write by NiFi UnfocusArchiveEntry
+  EXTENSIONAPI static constexpr const char* Description = "Restores a FlowFile which has had an archive entry focused via FocusArchiveEntry to its original state.";
+
+  static auto properties() { return std::array<core::Property, 0>{}; }
+
+  EXTENSIONAPI static const core::Relationship Success;
+  static auto relationships() { return std::array{Success}; }
+
+  EXTENSIONAPI static constexpr bool SupportsDynamicProperties = false;
+  EXTENSIONAPI static constexpr bool SupportsDynamicRelationships = false;
+  EXTENSIONAPI static constexpr core::annotation::Input InputRequirement = core::annotation::Input::INPUT_ALLOWED;
+  EXTENSIONAPI static constexpr bool IsSingleThreaded = false;
+
+  ADD_COMMON_VIRTUAL_FUNCTIONS_FOR_PROCESSORS
+
+  void onTrigger(core::ProcessContext *context, core::ProcessSession *session) override;
   void initialize() override;
 
   //! Write callback for reconstituting lensed archive into flow file content
@@ -77,7 +78,6 @@ class UnfocusArchiveEntry : public core::Processor {
   };
 
  private:
-  //! Logger
   std::shared_ptr<Logger> logger_ = core::logging::LoggerFactory<UnfocusArchiveEntry>::getLogger();
 };
 
diff --git a/extensions/librdkafka/ConsumeKafka.cpp b/extensions/librdkafka/ConsumeKafka.cpp
index 5182db4bb..809fd5b24 100644
--- a/extensions/librdkafka/ConsumeKafka.cpp
+++ b/extensions/librdkafka/ConsumeKafka.cpp
@@ -20,7 +20,6 @@
 #include <algorithm>
 #include <limits>
 
-#include "controllers/SSLContextService.h"
 #include "core/ProcessSession.h"
 #include "core/PropertyValidation.h"
 #include "core/Resource.h"
@@ -30,169 +29,28 @@
 
 using namespace std::literals::chrono_literals;
 
-namespace org {
-namespace apache {
-namespace nifi {
-namespace minifi {
+namespace org::apache::nifi::minifi {
+
 namespace core {
 // The upper limit for Max Poll Time is 4 seconds. This is because Watchdog would potentially start
 // reporting issues with the processor health otherwise
-class ConsumeKafkaMaxPollTimeValidator : public TimePeriodValidator {
- public:
-  ConsumeKafkaMaxPollTimeValidator(const std::string &name) // NOLINT
-      : TimePeriodValidator(name) {
-  }
-  ~ConsumeKafkaMaxPollTimeValidator() override = default;
+ConsumeKafkaMaxPollTimeValidator::ConsumeKafkaMaxPollTimeValidator(const std::string &name)
+  : TimePeriodValidator(name) {
+}
 
-  ValidationResult validate(const std::string& subject, const std::string& input) const override {
-    auto parsed_value = utils::timeutils::StringToDuration<std::chrono::milliseconds>(input);
-    return ValidationResult::Builder::createBuilder().withSubject(subject).withInput(input).isValid(
-        parsed_value.has_value() &&
-        0ms < *parsed_value && *parsed_value <= 4s).build();
-  }
-};
+ValidationResult ConsumeKafkaMaxPollTimeValidator::validate(const std::string& subject, const std::string& input) const {
+  auto parsed_value = utils::timeutils::StringToDuration<std::chrono::milliseconds>(input);
+  return ValidationResult::Builder::createBuilder().withSubject(subject).withInput(input).isValid(
+      parsed_value.has_value() &&
+      0ms < *parsed_value && *parsed_value <= 4s).build();
+}
 }  // namespace core
-namespace processors {
 
-constexpr const std::size_t ConsumeKafka::DEFAULT_MAX_POLL_RECORDS;
-constexpr char const* ConsumeKafka::DEFAULT_MAX_POLL_TIME;
-
-constexpr char const* ConsumeKafka::TOPIC_FORMAT_NAMES;
-constexpr char const* ConsumeKafka::TOPIC_FORMAT_PATTERNS;
-
-core::Property ConsumeKafka::KafkaBrokers(core::PropertyBuilder::createProperty("Kafka Brokers")
-  ->withDescription("A comma-separated list of known Kafka Brokers in the format <host>:<port>.")
-  ->withDefaultValue("localhost:9092", core::StandardValidators::get().NON_BLANK_VALIDATOR)
-  ->supportsExpressionLanguage(true)
-  ->isRequired(true)
-  ->build());
-
-core::Property ConsumeKafka::TopicNames(core::PropertyBuilder::createProperty("Topic Names")
-  ->withDescription("The name of the Kafka Topic(s) to pull from. Multiple topic names are supported as a comma separated list.")
-  ->supportsExpressionLanguage(true)
-  ->isRequired(true)
-  ->build());
-
-core::Property ConsumeKafka::TopicNameFormat(core::PropertyBuilder::createProperty("Topic Name Format")
-  ->withDescription("Specifies whether the Topic(s) provided are a comma separated list of names or a single regular expression. "
-                    "Using regular expressions does not automatically discover Kafka topics created after the processor started.")
-  ->withAllowableValues<std::string>({TOPIC_FORMAT_NAMES, TOPIC_FORMAT_PATTERNS})
-  ->withDefaultValue(TOPIC_FORMAT_NAMES)
-  ->isRequired(true)
-  ->build());
-
-core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions")
-  ->withDescription(
-      "Specifies whether or not MiNiFi should honor transactional guarantees when communicating with Kafka. If false, the Processor will use an \"isolation level\" of "
-      "read_uncomitted. This means that messages will be received as soon as they are written to Kafka but will be pulled, even if the producer cancels the transactions. "
-      "If this value is true, MiNiFi will not receive any messages for which the producer's transaction was canceled, but this can result in some latency since the consumer "
-      "must wait for the producer to finish its entire transaction instead of pulling as the messages become available.")
-  ->withDefaultValue<bool>(true)
-  ->isRequired(true)
-  ->build());
-
-core::Property ConsumeKafka::GroupID(core::PropertyBuilder::createProperty("Group ID")
-  ->withDescription("A Group ID is used to identify consumers that are within the same consumer group. Corresponds to Kafka's 'group.id' property.")
-  ->supportsExpressionLanguage(true)
-  ->isRequired(true)
-  ->build());
-
-core::Property ConsumeKafka::OffsetReset(core::PropertyBuilder::createProperty("Offset Reset")
-  ->withDescription("Allows you to manage the condition when there is no initial offset in Kafka or if the current offset does not exist any more on the server (e.g. because that "
-      "data has been deleted). Corresponds to Kafka's 'auto.offset.reset' property.")
-  ->withAllowableValues<std::string>({OFFSET_RESET_EARLIEST, OFFSET_RESET_LATEST, OFFSET_RESET_NONE})
-  ->withDefaultValue(OFFSET_RESET_LATEST)
-  ->isRequired(true)
-  ->build());
-
-core::Property ConsumeKafka::KeyAttributeEncoding(core::PropertyBuilder::createProperty("Key Attribute Encoding")
-  ->withDescription("FlowFiles that are emitted have an attribute named 'kafka.key'. This property dictates how the value of the attribute should be encoded.")
-  ->withAllowableValues<std::string>({KEY_ATTR_ENCODING_UTF_8, KEY_ATTR_ENCODING_HEX})
-  ->withDefaultValue(KEY_ATTR_ENCODING_UTF_8)
-  ->isRequired(true)
-  ->build());
-
-core::Property ConsumeKafka::MessageDemarcator(core::PropertyBuilder::createProperty("Message Demarcator")
-  ->withDescription("Since KafkaConsumer receives messages in batches, you have an option to output FlowFiles which contains all Kafka messages in a single batch "
-      "for a given topic and partition and this property allows you to provide a string (interpreted as UTF-8) to use for demarcating apart multiple Kafka messages. "
-      "This is an optional property and if not provided each Kafka message received will result in a single FlowFile which time it is triggered. ")
-  ->supportsExpressionLanguage(true)
-  ->build());
-
-core::Property ConsumeKafka::MessageHeaderEncoding(core::PropertyBuilder::createProperty("Message Header Encoding")
-  ->withDescription("Any message header that is found on a Kafka message will be added to the outbound FlowFile as an attribute. This property indicates the Character Encoding "
-      "to use for deserializing the headers.")
-  ->withAllowableValues<std::string>({MSG_HEADER_ENCODING_UTF_8, MSG_HEADER_ENCODING_HEX})
-  ->withDefaultValue(MSG_HEADER_ENCODING_UTF_8)
-  ->build());
-
-core::Property ConsumeKafka::HeadersToAddAsAttributes(core::PropertyBuilder::createProperty("Headers To Add As Attributes")
-  ->withDescription("A comma separated list to match against all message headers. Any message header whose name matches an item from the list will be added to the FlowFile "
-      "as an Attribute. If not specified, no Header values will be added as FlowFile attributes. The behaviour on when multiple headers of the same name are present is set using "
-      "the DuplicateHeaderHandling attribute.")
-  ->build());
-
-core::Property ConsumeKafka::DuplicateHeaderHandling(core::PropertyBuilder::createProperty("Duplicate Header Handling")
-  ->withDescription("For headers to be added as attributes, this option specifies how to handle cases where multiple headers are present with the same key. "
-      "For example in case of receiving these two headers: \"Accept: text/html\" and \"Accept: application/xml\" and we want to attach the value of \"Accept\" "
-      "as a FlowFile attribute:\n"
-      " - \"Keep First\" attaches: \"Accept -> text/html\"\n"
-      " - \"Keep Latest\" attaches: \"Accept -> application/xml\"\n"
-      " - \"Comma-separated Merge\" attaches: \"Accept -> text/html, application/xml\"\n")
-  ->withAllowableValues<std::string>({MSG_HEADER_KEEP_FIRST, MSG_HEADER_KEEP_LATEST, MSG_HEADER_COMMA_SEPARATED_MERGE})
-  ->withDefaultValue(MSG_HEADER_KEEP_LATEST)  // Mirroring NiFi behaviour
-  ->build());
-
-core::Property ConsumeKafka::MaxPollRecords(core::PropertyBuilder::createProperty("Max Poll Records")
-  ->withDescription("Specifies the maximum number of records Kafka should return when polling each time the processor is triggered.")
-  ->withDefaultValue<unsigned int>(DEFAULT_MAX_POLL_RECORDS)
-  ->build());
-
-core::Property ConsumeKafka::MaxPollTime(core::PropertyBuilder::createProperty("Max Poll Time")
-  ->withDescription("Specifies the maximum amount of time the consumer can use for polling data from the brokers. "
-      "Polling is a blocking operation, so the upper limit of this value is specified in 4 seconds.")
-  ->withDefaultValue(DEFAULT_MAX_POLL_TIME, std::make_shared<core::ConsumeKafkaMaxPollTimeValidator>(std::string("ConsumeKafkaMaxPollTimeValidator")))
-  ->isRequired(true)
-  ->build());
-
-core::Property ConsumeKafka::SessionTimeout(core::PropertyBuilder::createProperty("Session Timeout")
-  ->withDescription("Client group session and failure detection timeout. The consumer sends periodic heartbeats "
-      "to indicate its liveness to the broker. If no hearts are received by the broker for a group member within "
-      "the session timeout, the broker will remove the consumer from the group and trigger a rebalance. "
-      "The allowed range is configured with the broker configuration properties group.min.session.timeout.ms and group.max.session.timeout.ms.")
-  ->withDefaultValue<core::TimePeriodValue>("60 seconds")
-  ->build());
-
-const core::Relationship ConsumeKafka::Success("success", "Incoming Kafka messages as flowfiles. Depending on the demarcation strategy, this can be one or multiple flowfiles per message.");
+namespace processors {
 
 void ConsumeKafka::initialize() {
-  setSupportedProperties({
-    SecurityProtocol,
-    SSLContextService,
-    KerberosServiceName,
-    KerberosPrincipal,
-    KerberosKeytabPath,
-    SASLMechanism,
-    Username,
-    Password,
-    KafkaBrokers,
-    TopicNames,
-    TopicNameFormat,
-    HonorTransactions,
-    GroupID,
-    OffsetReset,
-    KeyAttributeEncoding,
-    MessageDemarcator,
-    MessageHeaderEncoding,
-    HeadersToAddAsAttributes,
-    DuplicateHeaderHandling,
-    MaxPollRecords,
-    MaxPollTime,
-    SessionTimeout
-  });
-  setSupportedRelationships({
-    Success,
-  });
+  setSupportedProperties(properties());
+  setSupportedRelationships(relationships());
 }
 
 void ConsumeKafka::onSchedule(core::ProcessContext* context, core::ProcessSessionFactory* /* sessionFactory */) {
@@ -537,12 +395,5 @@ void ConsumeKafka::onTrigger(core::ProcessContext* /* context */, core::ProcessS
   process_pending_messages(*session);
 }
 
-REGISTER_RESOURCE(ConsumeKafka, "Consumes messages from Apache Kafka and transform them into MiNiFi FlowFiles. "
-    "The application should make sure that the processor is triggered at regular intervals, even if no messages are expected, "
-    "to serve any queued callbacks waiting to be called. Rebalancing can also only happen on trigger."); // NOLINT
-
 }  // namespace processors
-}  // namespace minifi
-}  // namespace nifi
-}  // namespace apache
-}  // namespace org
+}  // namespace org::apache::nifi::minifi
diff --git a/extensions/librdkafka/ConsumeKafka.h b/extensions/librdkafka/ConsumeKafka.h
index 208bd1988..4ff9fc78a 100644
--- a/extensions/librdkafka/ConsumeKafka.h
+++ b/extensions/librdkafka/ConsumeKafka.h
@@ -29,38 +29,72 @@
 #include "rdkafka.h"
 #include "rdkafka_utils.h"
 #include "KafkaConnection.h"
+#include "utils/ArrayUtils.h"
 
-namespace org {
-namespace apache {
-namespace nifi {
-namespace minifi {
+namespace org::apache::nifi::minifi {
 
 class FlowFileRecord;
 
+namespace core {
+class ConsumeKafkaMaxPollTimeValidator : public TimePeriodValidator {
+ public:
+  explicit ConsumeKafkaMaxPollTimeValidator(const std::string &name);
+  ~ConsumeKafkaMaxPollTimeValidator() override = default;
+
+  [[nodiscard]] ValidationResult validate(const std::string& subject, const std::string& input) const override;
+};
+}  // namespace core
+
 namespace processors {
 
 class ConsumeKafka : public KafkaProcessorBase {
  public:
-  EXTENSIONAPI static constexpr char const* ProcessorName = "ConsumeKafka";
-
-  // Supported Properties
-  EXTENSIONAPI static core::Property KafkaBrokers;
-  EXTENSIONAPI static core::Property TopicNames;
-  EXTENSIONAPI static core::Property TopicNameFormat;
-  EXTENSIONAPI static core::Property HonorTransactions;
-  EXTENSIONAPI static core::Property GroupID;
-  EXTENSIONAPI static core::Property OffsetReset;
-  EXTENSIONAPI static core::Property KeyAttributeEncoding;
-  EXTENSIONAPI static core::Property MessageDemarcator;
-  EXTENSIONAPI static core::Property MessageHeaderEncoding;
-  EXTENSIONAPI static core::Property HeadersToAddAsAttributes;
-  EXTENSIONAPI static core::Property DuplicateHeaderHandling;
-  EXTENSIONAPI static core::Property MaxPollRecords;
-  EXTENSIONAPI static core::Property MaxPollTime;
-  EXTENSIONAPI static core::Property SessionTimeout;
-
-  // Supported Relationships
+  EXTENSIONAPI static constexpr const char* Description = "Consumes messages from Apache Kafka and transform them into MiNiFi FlowFiles. "
+      "The application should make sure that the processor is triggered at regular intervals, even if no messages are expected, "
+      "to serve any queued callbacks waiting to be called. Rebalancing can also only happen on trigger.";
+
+  EXTENSIONAPI static const core::Property KafkaBrokers;
+  EXTENSIONAPI static const core::Property TopicNames;
+  EXTENSIONAPI static const core::Property TopicNameFormat;
+  EXTENSIONAPI static const core::Property HonorTransactions;
+  EXTENSIONAPI static const core::Property GroupID;
+  EXTENSIONAPI static const core::Property OffsetReset;
+  EXTENSIONAPI static const core::Property KeyAttributeEncoding;
+  EXTENSIONAPI static const core::Property MessageDemarcator;
+  EXTENSIONAPI static const core::Property MessageHeaderEncoding;
+  EXTENSIONAPI static const core::Property HeadersToAddAsAttributes;
+  EXTENSIONAPI static const core::Property DuplicateHeaderHandling;
+  EXTENSIONAPI static const core::Property MaxPollRecords;
+  EXTENSIONAPI static const core::Property MaxPollTime;
+  EXTENSIONAPI static const core::Property SessionTimeout;
+  static auto properties() {
+    return utils::array_cat(KafkaProcessorBase::properties(), std::array{
+      KafkaBrokers,
+      TopicNames,
+      TopicNameFormat,
+      HonorTransactions,
+      GroupID,
+      OffsetReset,
+      KeyAttributeEncoding,
+      MessageDemarcator,
+      MessageHeaderEncoding,
+      HeadersToAddAsAttributes,
+      DuplicateHeaderHandling,
+      MaxPollRecords,
+      MaxPollTime,
+      SessionTimeout
+    });
+  }
+
   EXTENSIONAPI static const core::Relationship Success;
+  static auto relationships() { return std::array{Success}; }
+
+  EXTENSIONAPI static constexpr bool SupportsDynamicProperties = true;
+  EXTENSIONAPI static constexpr bool SupportsDynamicRelationships = false;
+  EXTENSIONAPI static constexpr core::annotation::Input InputRequirement = core::annotation::Input::INPUT_FORBIDDEN;
+  EXTENSIONAPI static constexpr bool IsSingleThreaded = false;
+
+  ADD_COMMON_VIRTUAL_FUNCTIONS_FOR_PROCESSORS
 
   // Security Protocol allowable values
   static constexpr char const* SECURITY_PROTOCOL_PLAINTEXT = "plaintext";
@@ -104,25 +138,8 @@ class ConsumeKafka : public KafkaProcessorBase {
 
   ~ConsumeKafka() override = default;
 
- public:
-  bool supportsDynamicProperties() override {
-    return true;
-  }
-  /**
-   * Function that's executed when the processor is scheduled.
-   * @param context process context.
-   * @param sessionFactory process session factory that is used when creating
-   * ProcessSession objects.
-   */
   void onSchedule(core::ProcessContext* context, core::ProcessSessionFactory* /* sessionFactory */) override;
-  /**
-   * Execution trigger for the RetryFlowFile Processor
-   * @param context processor context
-   * @param session processor session reference.
-   */
   void onTrigger(core::ProcessContext* context, core::ProcessSession* session) override;
-
-  // Initialize, overwrite by NiFi RetryFlowFile
   void initialize() override;
 
  private:
@@ -141,10 +158,6 @@ class ConsumeKafka : public KafkaProcessorBase {
   void process_pending_messages(core::ProcessSession& session);
 
  private:
-  core::annotation::Input getInputRequirement() const override {
-    return core::annotation::Input::INPUT_FORBIDDEN;
-  }
-
   std::string kafka_brokers_;
   std::vector<std::string> topic_names_;
   std::string topic_name_format_;
@@ -172,7 +185,4 @@ class ConsumeKafka : public KafkaProcessorBase {
 };
 
 }  // namespace processors
-}  // namespace minifi
-}  // namespace nifi
-}  // namespace apache
-}  // namespace org
+}  // namespace org::apache::nifi::minifi
diff --git a/extensions/librdkafka/KafkaProcessorBase.cpp b/extensions/librdkafka/KafkaProcessorBase.cpp
index 027328891..1a3fd142d 100644
--- a/extensions/librdkafka/KafkaProcessorBase.cpp
+++ b/extensions/librdkafka/KafkaProcessorBase.cpp
@@ -1,7 +1,4 @@
 /**
- * @file KafkaProcessorBase.cpp
- * KafkaProcessorBase class implementation
- *
  * 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.
@@ -25,46 +22,6 @@
 
 namespace org::apache::nifi::minifi::processors {
 
-const core::Property KafkaProcessorBase::SecurityProtocol(
-        core::PropertyBuilder::createProperty("Security Protocol")
-        ->withDescription("Protocol used to communicate with brokers. Corresponds to Kafka's 'security.protocol' property.")
-        ->withDefaultValue<std::string>(toString(SecurityProtocolOption::PLAINTEXT))
-        ->withAllowableValues<std::string>(SecurityProtocolOption::values())
-        ->isRequired(true)
-        ->build());
-const core::Property KafkaProcessorBase::SSLContextService(
-    core::PropertyBuilder::createProperty("SSL Context Service")
-        ->withDescription("SSL Context Service Name")
-        ->asType<minifi::controllers::SSLContextService>()
-        ->build());
-const core::Property KafkaProcessorBase::KerberosServiceName(
-    core::PropertyBuilder::createProperty("Kerberos Service Name")
-        ->withDescription("Kerberos Service Name")
-        ->build());
-const core::Property KafkaProcessorBase::KerberosPrincipal(
-    core::PropertyBuilder::createProperty("Kerberos Principal")
-        ->withDescription("Keberos Principal")
-        ->build());
-const core::Property KafkaProcessorBase::KerberosKeytabPath(
-    core::PropertyBuilder::createProperty("Kerberos Keytab Path")
-        ->withDescription("The path to the location on the local filesystem where the kerberos keytab is located. Read permission on the file is required.")
-        ->build());
-const core::Property KafkaProcessorBase::SASLMechanism(
-        core::PropertyBuilder::createProperty("SASL Mechanism")
-        ->withDescription("The SASL mechanism to use for authentication. Corresponds to Kafka's 'sasl.mechanism' property.")
-        ->withDefaultValue<std::string>(toString(SASLMechanismOption::GSSAPI))
-        ->withAllowableValues<std::string>(SASLMechanismOption::values())
-        ->isRequired(true)
-        ->build());
-const core::Property KafkaProcessorBase::Username(
-    core::PropertyBuilder::createProperty("Username")
-        ->withDescription("The username when the SASL Mechanism is sasl_plaintext")
-        ->build());
-const core::Property KafkaProcessorBase::Password(
-    core::PropertyBuilder::createProperty("Password")
-        ->withDescription("The password for the given username when the SASL Mechanism is sasl_plaintext")
-        ->build());
-
 std::optional<utils::SSL_data> KafkaProcessorBase::getSslData(core::ProcessContext& context) const {
   std::string ssl_service_name;
   if (context.getProperty(SSLContextService.getName(), ssl_service_name) && !ssl_service_name.empty()) {
diff --git a/extensions/librdkafka/KafkaProcessorBase.h b/extensions/librdkafka/KafkaProcessorBase.h
index 6879de20e..ab4b4b222 100644
--- a/extensions/librdkafka/KafkaProcessorBase.h
+++ b/extensions/librdkafka/KafkaProcessorBase.h
@@ -1,7 +1,4 @@
 /**
- * @file KafkaProcessorBase.h
- * KafkaProcessorBase class declaration
- *
  * 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.
@@ -29,7 +26,6 @@
 
 namespace org::apache::nifi::minifi::processors {
 
-// PublishKafka Class
 class KafkaProcessorBase : public core::Processor {
  public:
   EXTENSIONAPI static const core::Property SSLContextService;
@@ -40,6 +36,18 @@ class KafkaProcessorBase : public core::Processor {
   EXTENSIONAPI static const core::Property SASLMechanism;
   EXTENSIONAPI static const core::Property Username;
   EXTENSIONAPI static const core::Property Password;
+  static auto properties() {
+    return std::array{
+      SSLContextService,
+      SecurityProtocol,
+      KerberosServiceName,
+      KerberosPrincipal,
+      KerberosKeytabPath,
+      SASLMechanism,
+      Username,
+      Password
+    };
+  }
 
   SMART_ENUM(SecurityProtocolOption,
     (PLAINTEXT, "plaintext"),
diff --git a/extensions/librdkafka/KafkaProcessorStaticDefinitions.cpp b/extensions/librdkafka/KafkaProcessorStaticDefinitions.cpp
new file mode 100644
index 000000000..5b3ba5f94
--- /dev/null
+++ b/extensions/librdkafka/KafkaProcessorStaticDefinitions.cpp
@@ -0,0 +1,277 @@
+/**
+ * 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 "ConsumeKafka.h"
+#include "KafkaProcessorBase.h"
+#include "PublishKafka.h"
+#include "controllers/SSLContextService.h"
+#include "core/PropertyBuilder.h"
+#include "core/Resource.h"
+
+// FIXME(fgerlits): we need to put all these static definitions in a single file so that they are executed in this order at runtime
+// when https://issues.apache.org/jira/browse/MINIFICPP-1825 is closed, these definitions should be moved back to the cpp file of the class to which they belong
+
+namespace org::apache::nifi::minifi::processors {
+
+// KafkaProcessorBase
+
+const core::Property KafkaProcessorBase::SecurityProtocol(
+        core::PropertyBuilder::createProperty("Security Protocol")
+        ->withDescription("Protocol used to communicate with brokers. Corresponds to Kafka's 'security.protocol' property.")
+        ->withDefaultValue<std::string>(toString(SecurityProtocolOption::PLAINTEXT))
+        ->withAllowableValues<std::string>(SecurityProtocolOption::values())
+        ->isRequired(true)
+        ->build());
+const core::Property KafkaProcessorBase::SSLContextService(
+    core::PropertyBuilder::createProperty("SSL Context Service")
+        ->withDescription("SSL Context Service Name")
+        ->asType<minifi::controllers::SSLContextService>()
+        ->build());
+const core::Property KafkaProcessorBase::KerberosServiceName(
+    core::PropertyBuilder::createProperty("Kerberos Service Name")
+        ->withDescription("Kerberos Service Name")
+        ->build());
+const core::Property KafkaProcessorBase::KerberosPrincipal(
+    core::PropertyBuilder::createProperty("Kerberos Principal")
+        ->withDescription("Keberos Principal")
+        ->build());
+const core::Property KafkaProcessorBase::KerberosKeytabPath(
+    core::PropertyBuilder::createProperty("Kerberos Keytab Path")
+        ->withDescription("The path to the location on the local filesystem where the kerberos keytab is located. Read permission on the file is required.")
+        ->build());
+const core::Property KafkaProcessorBase::SASLMechanism(
+        core::PropertyBuilder::createProperty("SASL Mechanism")
+        ->withDescription("The SASL mechanism to use for authentication. Corresponds to Kafka's 'sasl.mechanism' property.")
+        ->withDefaultValue<std::string>(toString(SASLMechanismOption::GSSAPI))
+        ->withAllowableValues<std::string>(SASLMechanismOption::values())
+        ->isRequired(true)
+        ->build());
+const core::Property KafkaProcessorBase::Username(
+    core::PropertyBuilder::createProperty("Username")
+        ->withDescription("The username when the SASL Mechanism is sasl_plaintext")
+        ->build());
+const core::Property KafkaProcessorBase::Password(
+    core::PropertyBuilder::createProperty("Password")
+        ->withDescription("The password for the given username when the SASL Mechanism is sasl_plaintext")
+        ->build());
+
+
+// ConsumeKafka
+
+const core::Property ConsumeKafka::KafkaBrokers(core::PropertyBuilder::createProperty("Kafka Brokers")
+  ->withDescription("A comma-separated list of known Kafka Brokers in the format <host>:<port>.")
+  ->withDefaultValue("localhost:9092", core::StandardValidators::get().NON_BLANK_VALIDATOR)
+  ->supportsExpressionLanguage(true)
+  ->isRequired(true)
+  ->build());
+
+const core::Property ConsumeKafka::TopicNames(core::PropertyBuilder::createProperty("Topic Names")
+  ->withDescription("The name of the Kafka Topic(s) to pull from. Multiple topic names are supported as a comma separated list.")
+  ->supportsExpressionLanguage(true)
+  ->isRequired(true)
+  ->build());
+
+const core::Property ConsumeKafka::TopicNameFormat(core::PropertyBuilder::createProperty("Topic Name Format")
+  ->withDescription("Specifies whether the Topic(s) provided are a comma separated list of names or a single regular expression. "
+                    "Using regular expressions does not automatically discover Kafka topics created after the processor started.")
+  ->withAllowableValues<std::string>({TOPIC_FORMAT_NAMES, TOPIC_FORMAT_PATTERNS})
+  ->withDefaultValue(TOPIC_FORMAT_NAMES)
+  ->isRequired(true)
+  ->build());
+
+const core::Property ConsumeKafka::HonorTransactions(core::PropertyBuilder::createProperty("Honor Transactions")
+  ->withDescription(
+      "Specifies whether or not MiNiFi should honor transactional guarantees when communicating with Kafka. If false, the Processor will use an \"isolation level\" of "
+      "read_uncomitted. This means that messages will be received as soon as they are written to Kafka but will be pulled, even if the producer cancels the transactions. "
+      "If this value is true, MiNiFi will not receive any messages for which the producer's transaction was canceled, but this can result in some latency since the consumer "
+      "must wait for the producer to finish its entire transaction instead of pulling as the messages become available.")
+  ->withDefaultValue<bool>(true)
+  ->isRequired(true)
+  ->build());
+
+const core::Property ConsumeKafka::GroupID(core::PropertyBuilder::createProperty("Group ID")
+  ->withDescription("A Group ID is used to identify consumers that are within the same consumer group. Corresponds to Kafka's 'group.id' property.")
+  ->supportsExpressionLanguage(true)
+  ->isRequired(true)
+  ->build());
+
+const core::Property ConsumeKafka::OffsetReset(core::PropertyBuilder::createProperty("Offset Reset")
+  ->withDescription("Allows you to manage the condition when there is no initial offset in Kafka or if the current offset does not exist any more on the server (e.g. because that "
+      "data has been deleted). Corresponds to Kafka's 'auto.offset.reset' property.")
+  ->withAllowableValues<std::string>({OFFSET_RESET_EARLIEST, OFFSET_RESET_LATEST, OFFSET_RESET_NONE})
+  ->withDefaultValue(OFFSET_RESET_LATEST)
+  ->isRequired(true)
+  ->build());
+
+const core::Property ConsumeKafka::KeyAttributeEncoding(core::PropertyBuilder::createProperty("Key Attribute Encoding")
+  ->withDescription("FlowFiles that are emitted have an attribute named 'kafka.key'. This property dictates how the value of the attribute should be encoded.")
+  ->withAllowableValues<std::string>({KEY_ATTR_ENCODING_UTF_8, KEY_ATTR_ENCODING_HEX})
+  ->withDefaultValue(KEY_ATTR_ENCODING_UTF_8)
+  ->isRequired(true)
+  ->build());
+
+const core::Property ConsumeKafka::MessageDemarcator(core::PropertyBuilder::createProperty("Message Demarcator")
+  ->withDescription("Since KafkaConsumer receives messages in batches, you have an option to output FlowFiles which contains all Kafka messages in a single batch "
+      "for a given topic and partition and this property allows you to provide a string (interpreted as UTF-8) to use for demarcating apart multiple Kafka messages. "
+      "This is an optional property and if not provided each Kafka message received will result in a single FlowFile which time it is triggered. ")
+  ->supportsExpressionLanguage(true)
+  ->build());
+
+const core::Property ConsumeKafka::MessageHeaderEncoding(core::PropertyBuilder::createProperty("Message Header Encoding")
+  ->withDescription("Any message header that is found on a Kafka message will be added to the outbound FlowFile as an attribute. This property indicates the Character Encoding "
+      "to use for deserializing the headers.")
+  ->withAllowableValues<std::string>({MSG_HEADER_ENCODING_UTF_8, MSG_HEADER_ENCODING_HEX})
+  ->withDefaultValue(MSG_HEADER_ENCODING_UTF_8)
+  ->build());
+
+const core::Property ConsumeKafka::HeadersToAddAsAttributes(core::PropertyBuilder::createProperty("Headers To Add As Attributes")
+  ->withDescription("A comma separated list to match against all message headers. Any message header whose name matches an item from the list will be added to the FlowFile "
+      "as an Attribute. If not specified, no Header values will be added as FlowFile attributes. The behaviour on when multiple headers of the same name are present is set using "
+      "the DuplicateHeaderHandling attribute.")
+  ->build());
+
+const core::Property ConsumeKafka::DuplicateHeaderHandling(core::PropertyBuilder::createProperty("Duplicate Header Handling")
+  ->withDescription("For headers to be added as attributes, this option specifies how to handle cases where multiple headers are present with the same key. "
+      "For example in case of receiving these two headers: \"Accept: text/html\" and \"Accept: application/xml\" and we want to attach the value of \"Accept\" "
+      "as a FlowFile attribute:\n"
+      " - \"Keep First\" attaches: \"Accept -> text/html\"\n"
+      " - \"Keep Latest\" attaches: \"Accept -> application/xml\"\n"
+      " - \"Comma-separated Merge\" attaches: \"Accept -> text/html, application/xml\"\n")
+  ->withAllowableValues<std::string>({MSG_HEADER_KEEP_FIRST, MSG_HEADER_KEEP_LATEST, MSG_HEADER_COMMA_SEPARATED_MERGE})
+  ->withDefaultValue(MSG_HEADER_KEEP_LATEST)  // Mirroring NiFi behaviour
+  ->build());
+
+const core::Property ConsumeKafka::MaxPollRecords(core::PropertyBuilder::createProperty("Max Poll Records")
+  ->withDescription("Specifies the maximum number of records Kafka should return when polling each time the processor is triggered.")
+  ->withDefaultValue<unsigned int>(DEFAULT_MAX_POLL_RECORDS)
+  ->build());
+
+const core::Property ConsumeKafka::MaxPollTime(core::PropertyBuilder::createProperty("Max Poll Time")
+  ->withDescription("Specifies the maximum amount of time the consumer can use for polling data from the brokers. "
+      "Polling is a blocking operation, so the upper limit of this value is specified in 4 seconds.")
+  ->withDefaultValue(DEFAULT_MAX_POLL_TIME, std::make_shared<core::ConsumeKafkaMaxPollTimeValidator>(std::string("ConsumeKafkaMaxPollTimeValidator")))
+  ->isRequired(true)
+  ->build());
+
+const core::Property ConsumeKafka::SessionTimeout(core::PropertyBuilder::createProperty("Session Timeout")
+  ->withDescription("Client group session and failure detection timeout. The consumer sends periodic heartbeats "
+      "to indicate its liveness to the broker. If no hearts are received by the broker for a group member within "
+      "the session timeout, the broker will remove the consumer from the group and trigger a rebalance. "
+      "The allowed range is configured with the broker configuration properties group.min.session.timeout.ms and group.max.session.timeout.ms.")
+  ->withDefaultValue<core::TimePeriodValue>("60 seconds")
+  ->build());
+
+const core::Relationship ConsumeKafka::Success("success", "Incoming Kafka messages as flowfiles. Depending on the demarcation strategy, this can be one or multiple flowfiles per message.");
+
+REGISTER_RESOURCE(ConsumeKafka, Processor);
+
+
+// PublishKafka
+
+const core::Property PublishKafka::SeedBrokers(
+    core::PropertyBuilder::createProperty("Known Brokers")->withDescription("A comma-separated list of known Kafka Brokers in the format <host>:<port>")
+        ->isRequired(true)->supportsExpressionLanguage(true)->build());
+const core::Property PublishKafka::Topic(
+    core::PropertyBuilder::createProperty("Topic Name")->withDescription("The Kafka Topic of interest")
+        ->isRequired(true)->supportsExpressionLanguage(true)->build());
+
+const core::Property PublishKafka::DeliveryGuarantee(
+    core::PropertyBuilder::createProperty("Delivery Guarantee")->withDescription("Specifies the requirement for guaranteeing that a message is sent to Kafka. "
+                                                                                 "Valid values are 0 (do not wait for acks), "
+                                                                                 "-1 or all (block until message is committed by all in sync replicas) "
+                                                                                 "or any concrete number of nodes.")
+        ->isRequired(false)->supportsExpressionLanguage(true)->withDefaultValue(DELIVERY_ONE_NODE)->build());
+const core::Property PublishKafka::MaxMessageSize(
+    core::PropertyBuilder::createProperty("Max Request Size")->withDescription("Maximum Kafka protocol request message size")
+        ->isRequired(false)->build());
+
+const core::Property PublishKafka::RequestTimeOut(
+    core::PropertyBuilder::createProperty("Request Timeout")->withDescription("The ack timeout of the producer request")
+        ->isRequired(false)->withDefaultValue<core::TimePeriodValue>("10 sec")->build());
+
+const core::Property PublishKafka::MessageTimeOut(
+    core::PropertyBuilder::createProperty("Message Timeout")->withDescription("The total time sending a message could take")
+        ->isRequired(false)->withDefaultValue<core::TimePeriodValue>("30 sec")->build());
+
+const core::Property PublishKafka::ClientName(
+    core::PropertyBuilder::createProperty("Client Name")->withDescription("Client Name to use when communicating with Kafka")
+        ->isRequired(true)->supportsExpressionLanguage(true)->build());
+
+const core::Property PublishKafka::BatchSize(
+    core::PropertyBuilder::createProperty("Batch Size")->withDescription("Maximum number of messages batched in one MessageSet")
+        ->isRequired(false)->withDefaultValue<uint32_t>(10)->build());
+const core::Property PublishKafka::TargetBatchPayloadSize(
+    core::PropertyBuilder::createProperty("Target Batch Payload Size")->withDescription("The target total payload size for a batch. 0 B means unlimited (Batch Size is still applied).")
+        ->isRequired(false)->withDefaultValue<core::DataSizeValue>("512 KB")->build());
+const core::Property PublishKafka::AttributeNameRegex("Attributes to Send as Headers", "Any attribute whose name matches the regex will be added to the Kafka messages as a Header", "");
+
+const core::Property PublishKafka::QueueBufferMaxTime(
+        core::PropertyBuilder::createProperty("Queue Buffering Max Time")
+        ->isRequired(false)
+        ->withDefaultValue<core::TimePeriodValue>("5 millis")
+        ->withDescription("Delay to wait for messages in the producer queue to accumulate before constructing message batches")
+        ->build());
+const core::Property PublishKafka::QueueBufferMaxSize(
+        core::PropertyBuilder::createProperty("Queue Max Buffer Size")
+        ->isRequired(false)
+        ->withDefaultValue<core::DataSizeValue>("1 MB")
+        ->withDescription("Maximum total message size sum allowed on the producer queue")
+        ->build());
+const core::Property PublishKafka::QueueBufferMaxMessage(
+        core::PropertyBuilder::createProperty("Queue Max Message")
+        ->isRequired(false)
+        ->withDefaultValue<uint64_t>(1000)
+        ->withDescription("Maximum number of messages allowed on the producer queue")
+        ->build());
+const core::Property PublishKafka::CompressCodec(
+        core::PropertyBuilder::createProperty("Compress Codec")
+        ->isRequired(false)
+        ->withDefaultValue<std::string>(COMPRESSION_CODEC_NONE)
+        ->withAllowableValues<std::string>({COMPRESSION_CODEC_NONE, COMPRESSION_CODEC_GZIP, COMPRESSION_CODEC_SNAPPY})
+        ->withDescription("compression codec to use for compressing message sets")
+        ->build());
+const core::Property PublishKafka::MaxFlowSegSize(
+    core::PropertyBuilder::createProperty("Max Flow Segment Size")->withDescription("Maximum flow content payload segment size for the kafka record. 0 B means unlimited.")
+        ->isRequired(false)->withDefaultValue<core::DataSizeValue>("0 B")->build());
+
+const core::Property PublishKafka::SecurityCA("Security CA", "DEPRECATED in favor of SSL Context Service. File or directory path to CA certificate(s) for verifying the broker's key", "");
+const core::Property PublishKafka::SecurityCert("Security Cert", "DEPRECATED in favor of SSL Context Service.Path to client's public key (PEM) used for authentication", "");
+const core::Property PublishKafka::SecurityPrivateKey("Security Private Key", "DEPRECATED in favor of SSL Context Service.Path to client's private key (PEM) used for authentication", "");
+const core::Property PublishKafka::SecurityPrivateKeyPassWord("Security Pass Phrase", "DEPRECATED in favor of SSL Context Service.Private key passphrase", "");
+const core::Property PublishKafka::KafkaKey(
+    core::PropertyBuilder::createProperty("Kafka Key")
+        ->withDescription("The key to use for the message. If not specified, the UUID of the flow file is used as the message key.")
+        ->supportsExpressionLanguage(true)
+        ->build());
+const core::Property PublishKafka::MessageKeyField("Message Key Field", "DEPRECATED, does not work -- use Kafka Key instead", "");
+
+const core::Property PublishKafka::DebugContexts("Debug contexts", "A comma-separated list of debug contexts to enable."
+                                           "Including: generic, broker, topic, metadata, feature, queue, msg, protocol, cgrp, security, fetch, interceptor, plugin, consumer, admin, eos, all", "");
+const core::Property PublishKafka::FailEmptyFlowFiles(
+    core::PropertyBuilder::createProperty("Fail empty flow files")
+        ->withDescription("Keep backwards compatibility with <=0.7.0 bug which caused flow files with empty content to not be published to Kafka and forwarded to failure. The old behavior is "
+                          "deprecated. Use connections to drop empty flow files!")
+        ->isRequired(false)
+        ->withDefaultValue<bool>(true)
+        ->build());
+
+const core::Relationship PublishKafka::Success("success", "Any FlowFile that is successfully sent to Kafka will be routed to this Relationship");
+const core::Relationship PublishKafka::Failure("failure", "Any FlowFile that cannot be sent to Kafka will be routed to this Relationship");
+
+REGISTER_RESOURCE(PublishKafka, Processor);
+
+}  // namespace org::apache::nifi::minifi::processors
diff --git a/extensions/librdkafka/PublishKafka.cpp b/extensions/librdkafka/PublishKafka.cpp
index 72756be72..3516c7dcd 100644
--- a/extensions/librdkafka/PublishKafka.cpp
+++ b/extensions/librdkafka/PublishKafka.cpp
@@ -1,7 +1,4 @@
 /**
- * @file PublishKafka.cpp
- * PublishKafka class implementation
- *
  * 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.
@@ -29,108 +26,12 @@
 #include <vector>
 
 #include "utils/gsl.h"
-#include "utils/TimeUtil.h"
 #include "utils/StringUtils.h"
 #include "core/ProcessContext.h"
 #include "core/ProcessSession.h"
 #include "core/Resource.h"
 
-namespace org {
-namespace apache {
-namespace nifi {
-namespace minifi {
-namespace processors {
-
-const core::Property PublishKafka::SeedBrokers(
-    core::PropertyBuilder::createProperty("Known Brokers")->withDescription("A comma-separated list of known Kafka Brokers in the format <host>:<port>")
-        ->isRequired(true)->supportsExpressionLanguage(true)->build());
-const core::Property PublishKafka::Topic(
-    core::PropertyBuilder::createProperty("Topic Name")->withDescription("The Kafka Topic of interest")
-        ->isRequired(true)->supportsExpressionLanguage(true)->build());
-
-const core::Property PublishKafka::DeliveryGuarantee(
-    core::PropertyBuilder::createProperty("Delivery Guarantee")->withDescription("Specifies the requirement for guaranteeing that a message is sent to Kafka. "
-                                                                                 "Valid values are 0 (do not wait for acks), "
-                                                                                 "-1 or all (block until message is committed by all in sync replicas) "
-                                                                                 "or any concrete number of nodes.")
-        ->isRequired(false)->supportsExpressionLanguage(true)->withDefaultValue(DELIVERY_ONE_NODE)->build());
-const core::Property PublishKafka::MaxMessageSize(
-    core::PropertyBuilder::createProperty("Max Request Size")->withDescription("Maximum Kafka protocol request message size")
-        ->isRequired(false)->build());
-
-const core::Property PublishKafka::RequestTimeOut(
-    core::PropertyBuilder::createProperty("Request Timeout")->withDescription("The ack timeout of the producer request")
-        ->isRequired(false)->withDefaultValue<core::TimePeriodValue>("10 sec")->build());
-
-const core::Property PublishKafka::MessageTimeOut(
-    core::PropertyBuilder::createProperty("Message Timeout")->withDescription("The total time sending a message could take")
-        ->isRequired(false)->withDefaultValue<core::TimePeriodValue>("30 sec")->build());
-
-const core::Property PublishKafka::ClientName(
-    core::PropertyBuilder::createProperty("Client Name")->withDescription("Client Name to use when communicating with Kafka")
-        ->isRequired(true)->supportsExpressionLanguage(true)->build());
-
-const core::Property PublishKafka::BatchSize(
-    core::PropertyBuilder::createProperty("Batch Size")->withDescription("Maximum number of messages batched in one MessageSet")
-        ->isRequired(false)->withDefaultValue<uint32_t>(10)->build());
-const core::Property PublishKafka::TargetBatchPayloadSize(
-    core::PropertyBuilder::createProperty("Target Batch Payload Size")->withDescription("The target total payload size for a batch. 0 B means unlimited (Batch Size is still applied).")
-        ->isRequired(false)->withDefaultValue<core::DataSizeValue>("512 KB")->build());
-const core::Property PublishKafka::AttributeNameRegex("Attributes to Send as Headers", "Any attribute whose name matches the regex will be added to the Kafka messages as a Header", "");
-
-const core::Property PublishKafka::QueueBufferMaxTime(
-        core::PropertyBuilder::createProperty("Queue Buffering Max Time")
-        ->isRequired(false)
-        ->withDefaultValue<core::TimePeriodValue>("5 millis")
-        ->withDescription("Delay to wait for messages in the producer queue to accumulate before constructing message batches")
-        ->build());
-const core::Property PublishKafka::QueueBufferMaxSize(
-        core::PropertyBuilder::createProperty("Queue Max Buffer Size")
-        ->isRequired(false)
-        ->withDefaultValue<core::DataSizeValue>("1 MB")
-        ->withDescription("Maximum total message size sum allowed on the producer queue")
-        ->build());
-const core::Property PublishKafka::QueueBufferMaxMessage(
-        core::PropertyBuilder::createProperty("Queue Max Message")
-        ->isRequired(false)
-        ->withDefaultValue<uint64_t>(1000)
-        ->withDescription("Maximum number of messages allowed on the producer queue")
-        ->build());
-const core::Property PublishKafka::CompressCodec(
-        core::PropertyBuilder::createProperty("Compress Codec")
-        ->isRequired(false)
-        ->withDefaultValue<std::string>(COMPRESSION_CODEC_NONE)
-        ->withAllowableValues<std::string>({COMPRESSION_CODEC_NONE, COMPRESSION_CODEC_GZIP, COMPRESSION_CODEC_SNAPPY})
-        ->withDescription("compression codec to use for compressing message sets")
-        ->build());
-const core::Property PublishKafka::MaxFlowSegSize(
-    core::PropertyBuilder::createProperty("Max Flow Segment Size")->withDescription("Maximum flow content payload segment size for the kafka record. 0 B means unlimited.")
-        ->isRequired(false)->withDefaultValue<core::DataSizeValue>("0 B")->build());
-
-const core::Property PublishKafka::SecurityCA("Security CA", "DEPRECATED in favor of SSL Context Service. File or directory path to CA certificate(s) for verifying the broker's key", "");
-const core::Property PublishKafka::SecurityCert("Security Cert", "DEPRECATED in favor of SSL Context Service.Path to client's public key (PEM) used for authentication", "");
-const core::Property PublishKafka::SecurityPrivateKey("Security Private Key", "DEPRECATED in favor of SSL Context Service.Path to client's private key (PEM) used for authentication", "");
-const core::Property PublishKafka::SecurityPrivateKeyPassWord("Security Pass Phrase", "DEPRECATED in favor of SSL Context Service.Private key passphrase", "");
-const core::Property PublishKafka::KafkaKey(
-    core::PropertyBuilder::createProperty("Kafka Key")
-        ->withDescription("The key to use for the message. If not specified, the UUID of the flow file is used as the message key.")
-        ->supportsExpressionLanguage(true)
-        ->build());
-const core::Property PublishKafka::MessageKeyField("Message Key Field", "DEPRECATED, does not work -- use Kafka Key instead", "");
-
-const core::Property PublishKafka::DebugContexts("Debug contexts", "A comma-separated list of debug contexts to enable."
-                                           "Including: generic, broker, topic, metadata, feature, queue, msg, protocol, cgrp, security, fetch, interceptor, plugin, consumer, admin, eos, all", "");
-const core::Property PublishKafka::FailEmptyFlowFiles(
-    core::PropertyBuilder::createProperty("Fail empty flow files")
-        ->withDescription("Keep backwards compatibility with <=0.7.0 bug which caused flow files with empty content to not be published to Kafka and forwarded to failure. The old behavior is "
-                          "deprecated. Use connections to drop empty flow files!")
-        ->isRequired(false)
-        ->withDefaultValue<bool>(true)
-        ->build());
-
-const core::Relationship PublishKafka::Success("success", "Any FlowFile that is successfully sent to Kafka will be routed to this Relationship");
-const core::Relationship PublishKafka::Failure("failure", "Any FlowFile that cannot be sent to Kafka will be routed to this Relationship");
-
+namespace org::apache::nifi::minifi::processors {
 
 namespace {
 struct rd_kafka_conf_deleter {
@@ -440,45 +341,8 @@ void messageDeliveryCallback(rd_kafka_t* rk, const rd_kafka_message_t* rkmessage
 }  // namespace
 
 void PublishKafka::initialize() {
-  // Set the supported properties
-  setSupportedProperties({
-    SeedBrokers,
-    Topic,
-    DeliveryGuarantee,
-    MaxMessageSize,
-    RequestTimeOut,
-    MessageTimeOut,
-    ClientName,
-    AttributeNameRegex,
-    BatchSize,
-    TargetBatchPayloadSize,
-    QueueBufferMaxTime,
-    QueueBufferMaxSize,
-    QueueBufferMaxMessage,
-    CompressCodec,
-    MaxFlowSegSize,
-    SecurityProtocol,
-    SSLContextService,
-    SecurityCA,
-    SecurityCert,
-    SecurityPrivateKey,
-    SecurityPrivateKeyPassWord,
-    KerberosServiceName,
-    KerberosPrincipal,
-    KerberosKeytabPath,
-    KafkaKey,
-    MessageKeyField,
-    DebugContexts,
-    FailEmptyFlowFiles,
-    SASLMechanism,
-    Username,
-    Password
-  });
-  // Set the supported relationships
-  setSupportedRelationships({
-    Success,
-    Failure
-  });
+  setSupportedProperties(properties());
+  setSupportedRelationships(relationships());
 }
 
 void PublishKafka::onSchedule(const std::shared_ptr<core::ProcessContext> &context, const std::shared_ptr<core::ProcessSessionFactory>& /*sessionFactory*/) {
@@ -939,11 +803,4 @@ void PublishKafka::onTrigger(const std::shared_ptr<core::ProcessContext> &contex
   });
 }
 
-REGISTER_RESOURCE(PublishKafka, "This Processor puts the contents of a FlowFile to a Topic in Apache Kafka. The content of a FlowFile becomes the contents of a Kafka message. "
-                  "This message is optionally assigned a key by using the <Kafka Key> Property.");
-
-}  // namespace processors
-}  // namespace minifi
-}  // namespace nifi
-}  // namespace apache
-}  // namespace org
+}  // namespace org::apache::nifi::minifi::processors
diff --git a/extensions/librdkafka/PublishKafka.h b/extensions/librdkafka/PublishKafka.h
index 153cc1955..bbf4cc9fd 100644
--- a/extensions/librdkafka/PublishKafka.h
+++ b/extensions/librdkafka/PublishKafka.h
@@ -1,7 +1,4 @@
 /**
- * @file PublishKafka.h
- * PublishKafka class declaration
- *
  * 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.
@@ -42,20 +39,17 @@
 #include "controllers/SSLContextService.h"
 #include "rdkafka.h"
 #include "KafkaConnection.h"
+#include "utils/ArrayUtils.h"
 #include "utils/RegexUtils.h"
 
-namespace org {
-namespace apache {
-namespace nifi {
-namespace minifi {
-namespace processors {
+namespace org::apache::nifi::minifi::processors {
 
-// PublishKafka Class
 class PublishKafka : public KafkaProcessorBase {
  public:
-  static constexpr char const* ProcessorName = "PublishKafka";
+  EXTENSIONAPI static constexpr const char* Description = "This Processor puts the contents of a FlowFile to a Topic in Apache Kafka. "
+      "The content of a FlowFile becomes the contents of a Kafka message. "
+      "This message is optionally assigned a key by using the <Kafka Key> Property.";
 
-  // Supported Properties
   EXTENSIONAPI static const core::Property SeedBrokers;
   EXTENSIONAPI static const core::Property Topic;
   EXTENSIONAPI static const core::Property DeliveryGuarantee;
@@ -79,10 +73,44 @@ class PublishKafka : public KafkaProcessorBase {
   EXTENSIONAPI static const core::Property MessageKeyField;
   EXTENSIONAPI static const core::Property DebugContexts;
   EXTENSIONAPI static const core::Property FailEmptyFlowFiles;
+  static auto properties() {
+    return utils::array_cat(KafkaProcessorBase::properties(), std::array{
+      SeedBrokers,
+      Topic,
+      DeliveryGuarantee,
+      MaxMessageSize,
+      RequestTimeOut,
+      MessageTimeOut,
+      ClientName,
+      BatchSize,
+      TargetBatchPayloadSize,
+      AttributeNameRegex,
+      QueueBufferMaxTime,
+      QueueBufferMaxSize,
+      QueueBufferMaxMessage,
+      CompressCodec,
+      MaxFlowSegSize,
+      SecurityCA,
+      SecurityCert,
+      SecurityPrivateKey,
+      SecurityPrivateKeyPassWord,
+      KafkaKey,
+      MessageKeyField,
+      DebugContexts,
+      FailEmptyFlowFiles
+    });
+  }
 
-  // Supported Relationships
-  EXTENSIONAPI static const core::Relationship Failure;
   EXTENSIONAPI static const core::Relationship Success;
+  EXTENSIONAPI static const core::Relationship Failure;
+  static auto relationships() { return std::array{Success, Failure}; }
+
+  EXTENSIONAPI static constexpr bool SupportsDynamicProperties = true;
+  EXTENSIONAPI static constexpr bool SupportsDynamicRelationships = false;
+  EXTENSIONAPI static constexpr core::annotation::Input InputRequirement = core::annotation::Input::INPUT_REQUIRED;
+  EXTENSIONAPI static constexpr bool IsSingleThreaded = false;
+
+  ADD_COMMON_VIRTUAL_FUNCTIONS_FOR_PROCESSORS
 
   static constexpr const char* COMPRESSION_CODEC_NONE = "none";
   static constexpr const char* COMPRESSION_CODEC_GZIP = "gzip";
@@ -101,14 +129,6 @@ class PublishKafka : public KafkaProcessorBase {
 
   ~PublishKafka() override = default;
 
-  bool supportsDynamicProperties() override { return true; }
-
-  /**
-   * Function that's executed when the processor is scheduled.
-   * @param context process context.
-   * @param sessionFactory process session factory that is used when creating
-   * ProcessSession objects.
-   */
   void onTrigger(const std::shared_ptr<core::ProcessContext> &context, const std::shared_ptr<core::ProcessSession> &session) override;
   void initialize() override;
   void onSchedule(const std::shared_ptr<core::ProcessContext> &context, const std::shared_ptr<core::ProcessSessionFactory> &sessionFactory) override;
@@ -122,10 +142,6 @@ class PublishKafka : public KafkaProcessorBase {
   std::optional<utils::SSL_data> getSslData(core::ProcessContext& context) const override;
 
  private:
-  core::annotation::Input getInputRequirement() const override {
-    return core::annotation::Input::INPUT_REQUIRED;
-  }
-
   KafkaConnectionKey key_;
   std::unique_ptr<KafkaConnection> conn_;
   std::mutex connection_mutex_;
@@ -140,10 +156,6 @@ class PublishKafka : public KafkaProcessorBase {
   std::set<std::shared_ptr<Messages>> messages_set_;
 };
 
-}  // namespace processors
-}  // namespace minifi
-}  // namespace nifi
-}  // namespace apache
-}  // namespace org
+}  // namespace org::apache::nifi::minifi::processors
 
 #endif  // EXTENSIONS_LIBRDKAFKA_PUBLISHKAFKA_H_
diff --git a/extensions/mqtt/controllerservice/MQTTControllerService.cpp b/extensions/mqtt/controllerservice/MQTTControllerService.cpp
index c5333b5c9..aeb82bcc7 100644
--- a/extensions/mqtt/controllerservice/MQTTControllerService.cpp
+++ b/extensions/mqtt/controllerservice/MQTTControllerService.cpp
@@ -1,5 +1,4 @@
 /**
- *
  * 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.
@@ -22,27 +21,22 @@
 #include <openssl/ssl.h>
 #include <string>
 #include <memory>
-#include <set>
 #include "core/Property.h"
 #include "core/Resource.h"
 #include "io/validation.h"
 #include "properties/Configure.h"
 
-namespace org {
-namespace apache {
-namespace nifi {
-namespace minifi {
-namespace controllers {
+namespace org::apache::nifi::minifi::controllers {
 
-core::Property MQTTControllerService::BrokerURL("Broker URI", "The URI to use to connect to the MQTT broker", "");
-core::Property MQTTControllerService::ClientID("Client ID", "MQTT client ID to use", "");
-core::Property MQTTControllerService::UserName("Username", "Username to use when connecting to the broker", "");
-core::Property MQTTControllerService::Password("Password", "Password to use when connecting to the broker", "");
-core::Property MQTTControllerService::KeepLiveInterval("Keep Alive Interval", "Defines the maximum time interval between messages sent or received", "60 sec");
-core::Property MQTTControllerService::ConnectionTimeout("Connection Timeout", "Maximum time interval the client will wait for the network connection to the MQTT server", "30 sec");
-core::Property MQTTControllerService::QOS("Quality of Service", "The Quality of Service(QoS) to send the message with. Accepts three values '0', '1' and '2'", "MQTT_QOS_0");
-core::Property MQTTControllerService::Topic("Topic", "The topic to publish the message to", "");
-core::Property MQTTControllerService::SecurityProtocol("Security Protocol", "Protocol used to communicate with brokers", "");
+const core::Property MQTTControllerService::BrokerURL("Broker URI", "The URI to use to connect to the MQTT broker", "");
+const core::Property MQTTControllerService::ClientID("Client ID", "MQTT client ID to use", "");
+const core::Property MQTTControllerService::UserName("Username", "Username to use when connecting to the broker", "");
+const core::Property MQTTControllerService::Password("Password", "Password to use when connecting to the broker", "");
+const core::Property MQTTControllerService::KeepLiveInterval("Keep Alive Interval", "Defines the maximum time interval between messages sent or received", "60 sec");
+const core::Property MQTTControllerService::ConnectionTimeout("Connection Timeout", "Maximum time interval the client will wait for the network connection to the MQTT server", "30 sec");
+const core::Property MQTTControllerService::QOS("Quality of Service", "The Quality of Service(QoS) to send the message with. Accepts three values '0', '1' and '2'", "MQTT_QOS_0");
+const core::Property MQTTControllerService::Topic("Topic", "The topic to publish the message to", "");
+const core::Property MQTTControllerService::SecurityProtocol("Security Protocol", "Protocol used to communicate with brokers", "");
 
 void MQTTControllerService::initialize() {
   std::lock_guard<std::mutex> lock(initialization_mutex_);
@@ -79,24 +73,9 @@ void MQTTControllerService::onEnable() {
 }
 
 void MQTTControllerService::initializeProperties() {
-  std::set<core::Property> supportedProperties;
-  supportedProperties.insert(BrokerURL);
-  supportedProperties.insert(ClientID);
-  supportedProperties.insert(UserName);
-  supportedProperties.insert(Password);
-
-  supportedProperties.insert(KeepLiveInterval);
-  supportedProperties.insert(ConnectionTimeout);
-  supportedProperties.insert(Topic);
-  supportedProperties.insert(QOS);
-  supportedProperties.insert(SecurityProtocol);
-  setSupportedProperties(supportedProperties);
+  setSupportedProperties(properties());
 }
 
-REGISTER_INTERNAL_RESOURCE_AS(MQTTControllerService, ("MQTTContextService"));
+REGISTER_RESOURCE_AS(MQTTControllerService, InternalResource, ("MQTTContextService"));
 
-} /* namespace controllers */
-} /* namespace minifi */
-} /* namespace nifi */
-} /* namespace apache */
-} /* namespace org */
+}  // namespace org::apache::nifi::minifi::controllers
diff --git a/extensions/mqtt/controllerservice/MQTTControllerService.h b/extensions/mqtt/controllerservice/MQTTControllerService.h
index b7ca764d7..6fee09bab 100644
--- a/extensions/mqtt/controllerservice/MQTTControllerService.h
+++ b/extensions/mqtt/controllerservice/MQTTControllerService.h
@@ -1,5 +1,4 @@
 /**
- *
  * 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.
@@ -35,11 +34,7 @@
 #include "concurrentqueue.h"
 #include "MQTTClient.h"
 
-namespace org {
-namespace apache {
-namespace nifi {
-namespace minifi {
-namespace controllers {
+namespace org::apache::nifi::minifi::controllers {
 
 static constexpr const char* const MQTT_QOS_0 = "0";
 static constexpr const char* const MQTT_QOS_1 = "1";
@@ -90,20 +85,35 @@ class MQTTControllerService : public core::controller::ControllerService {
     initialize();
   }
 
-  static core::Property BrokerURL;
-  static core::Property ClientID;
-  static core::Property UserName;
-  static core::Property Password;
-  static core::Property CleanSession;
-  static core::Property KeepLiveInterval;
-  static core::Property ConnectionTimeout;
-  static core::Property Topic;
-  static core::Property QOS;
-  static core::Property SecurityProtocol;
+  EXTENSIONAPI static const core::Property BrokerURL;
+  EXTENSIONAPI static const core::Property ClientID;
+  EXTENSIONAPI static const core::Property UserName;
+  EXTENSIONAPI static const core::Property Password;
+  EXTENSIONAPI static const core::Property KeepLiveInterval;
+  EXTENSIONAPI static const core::Property ConnectionTimeout;
+  EXTENSIONAPI static const core::Property Topic;
+  EXTENSIONAPI static const core::Property QOS;
+  EXTENSIONAPI static const core::Property SecurityProtocol;
+  static auto properties() {
+    return std::array{
+      BrokerURL,
+      ClientID,
+      UserName,
+      Password,
+      KeepLiveInterval,
+      ConnectionTimeout,
+      Topic,
+      QOS,
+      SecurityProtocol
+    };
+  }
+
+  EXTENSIONAPI static constexpr bool SupportsDynamicProperties = false;
+  ADD_COMMON_VIRTUAL_FUNCTIONS_FOR_CONTROLLER_SERVICES
 
-  virtual void initialize();
+  void initialize() override;
 
-  void yield() {
+  void yield() override {
   }
 
   int send(const std::string &topic, const std::vector<uint8_t> &data) {
@@ -149,15 +159,15 @@ class MQTTControllerService : public core::controller::ControllerService {
     return token;
   }
 
-  bool isRunning() {
+  bool isRunning() override {
     return getState() == core::controller::ControllerServiceState::ENABLED;
   }
 
-  bool isWorkAvailable() {
+  bool isWorkAvailable() override {
     return false;
   }
 
-  virtual void onEnable();
+  void onEnable() override;
 
   void subscribeToTopic(const std::string newTopic) {
     std::lock_guard<std::mutex> lock(initialization_mutex_);
@@ -314,8 +324,4 @@ class MQTTControllerService : public core::controller::ControllerService {
   std::shared_ptr<core::logging::Logger> logger_ = core::logging::LoggerFactory<MQTTControllerService>::getLogger();
 };
 
-} /* namespace controllers */
-} /* namespace minifi */
-} /* namespace nifi */
-} /* namespace apache */
-} /* namespace org */
+}  // namespace org::apache::nifi::minifi::controllers
diff --git a/extensions/mqtt/processors/AbstractMQTTProcessor.cpp b/extensions/mqtt/processors/AbstractMQTTProcessor.cpp
index 8b838348e..b9bd11762 100644
--- a/extensions/mqtt/processors/AbstractMQTTProcessor.cpp
+++ b/extensions/mqtt/processors/AbstractMQTTProcessor.cpp
@@ -1,7 +1,4 @@
 /**
- * @file AbstractMQTTProcessor.cpp
- * AbstractMQTTProcessor class implementation
- *
  * 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.
@@ -22,37 +19,14 @@
 #include <memory>
 #include <string>
 #include <cinttypes>
-#include <set>
+#include <vector>
 
 #include "utils/TimeUtil.h"
 #include "utils/StringUtils.h"
 #include "core/ProcessContext.h"
 #include "core/ProcessSession.h"
 
-namespace org {
-namespace apache {
-namespace nifi {
-namespace minifi {
-namespace processors {
-
-core::Property AbstractMQTTProcessor::BrokerURL("Broker URI", "The URI to use to connect to the MQTT broker", "");
-core::Property AbstractMQTTProcessor::CleanSession("Session state", "Whether to start afresh or resume previous flows. See the allowable value descriptions for more details", "true");
-core::Property AbstractMQTTProcessor::ClientID("Client ID", "MQTT client ID to use", "");
-core::Property AbstractMQTTProcessor::UserName("Username", "Username to use when connecting to the broker", "");
-core::Property AbstractMQTTProcessor::PassWord("Password", "Password to use when connecting to the broker", "");
-core::Property AbstractMQTTProcessor::KeepLiveInterval("Keep Alive Interval", "Defines the maximum time interval between messages sent or received", "60 sec");
-core::Property AbstractMQTTProcessor::ConnectionTimeout("Connection Timeout", "Maximum time interval the client will wait for the network connection to the MQTT server", "30 sec");
-core::Property AbstractMQTTProcessor::QOS("Quality of Service", "The Quality of Service(QoS) to send the message with. Accepts three values '0', '1' and '2'", "MQTT_QOS_0");
-core::Property AbstractMQTTProcessor::Topic("Topic", "The topic to publish the message to", "");
-core::Property AbstractMQTTProcessor::SecurityProtocol("Security Protocol", "Protocol used to communicate with brokers", "");
-core::Property AbstractMQTTProcessor::SecurityCA("Security CA", "File or directory path to CA certificate(s) for verifying the broker's key", "");
-core::Property AbstractMQTTProcessor::SecurityCert("Security Cert", "Path to client's public key (PEM) used for authentication", "");
-core::Property AbstractMQTTProcessor::SecurityPrivateKey("Security Private Key", "Path to client's private key (PEM) used for authentication", "");
-core::Property AbstractMQTTProcessor::SecurityPrivateKeyPassWord("Security Pass Phrase", "Private key passphrase", "");
-
-const std::set<core::Property> AbstractMQTTProcessor::getSupportedProperties() {
-  return {BrokerURL, CleanSession, ClientID, UserName, PassWord, KeepLiveInterval, ConnectionTimeout, QOS, Topic};
-}
+namespace org::apache::nifi::minifi::processors {
 
 void AbstractMQTTProcessor::onSchedule(const std::shared_ptr<core::ProcessContext> &context, const std::shared_ptr<core::ProcessSessionFactory>& /*factory*/) {
   sslEnabled_ = false;
@@ -184,8 +158,4 @@ bool AbstractMQTTProcessor::reconnect() {
   return true;
 }
 
-} /* namespace processors */
-} /* namespace minifi */
-} /* namespace nifi */
-} /* namespace apache */
-} /* namespace org */
+}  // namespace org::apache::nifi::minifi::processors
diff --git a/extensions/mqtt/processors/AbstractMQTTProcessor.h b/extensions/mqtt/processors/AbstractMQTTProcessor.h
index 1b41f4cbe..b8e8e37d0 100644
--- a/extensions/mqtt/processors/AbstractMQTTProcessor.h
+++ b/extensions/mqtt/processors/AbstractMQTTProcessor.h
@@ -1,7 +1,4 @@
 /**
- * @file AbstractMQTTProcessor.h
- * AbstractMQTTProcessor class declaration
- *
  * 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.
@@ -19,9 +16,9 @@
  */
 #pragma once
 
-#include <set>
 #include <string>
 #include <memory>
+#include <vector>
 
 #include "FlowFileRecord.h"
 #include "core/Processor.h"
@@ -30,11 +27,7 @@
 #include "core/logging/LoggerConfiguration.h"
 #include "MQTTClient.h"
 
-namespace org {
-namespace apache {
-namespace nifi {
-namespace minifi {
-namespace processors {
+namespace org::apache::nifi::minifi::processors {
 
 static constexpr const char* const MQTT_QOS_0 = "0";
 static constexpr const char* const MQTT_QOS_1 = "1";
@@ -43,13 +36,8 @@ static constexpr const char* const MQTT_QOS_2 = "2";
 static constexpr const char* const MQTT_SECURITY_PROTOCOL_PLAINTEXT = "plaintext";
 static constexpr const char* const MQTT_SECURITY_PROTOCOL_SSL = "ssl";
 
-// AbstractMQTTProcessor Class
 class AbstractMQTTProcessor : public core::Processor {
  public:
-  // Constructor
-  /*!
-   * Create a new processor
-   */
   explicit AbstractMQTTProcessor(const std::string& name, const utils::Identifier& uuid = {})
       : core::Processor(name, uuid) {
     client_ = nullptr;
@@ -57,7 +45,7 @@ class AbstractMQTTProcessor : public core::Processor {
     qos_ = 0;
     isSubscriber_ = false;
   }
-  // Destructor
+
   ~AbstractMQTTProcessor() override {
     if (isSubscriber_) {
       MQTTClient_unsubscribe(client_, topic_.c_str());
@@ -68,31 +56,41 @@ class AbstractMQTTProcessor : public core::Processor {
     if (client_)
       MQTTClient_destroy(&client_);
   }
-  // Processor Name
-  static constexpr char const* ProcessorName = "AbstractMQTTProcessor";
-  // Supported Properties
-  static core::Property BrokerURL;
-  static core::Property ClientID;
-  static core::Property UserName;
-  static core::Property PassWord;
-  static core::Property CleanSession;
-  static core::Property KeepLiveInterval;
-  static core::Property ConnectionTimeout;
-  static core::Property Topic;
-  static core::Property QOS;
-  static core::Property SecurityProtocol;
-  static core::Property SecurityCA;
-  static core::Property SecurityCert;
-  static core::Property SecurityPrivateKey;
-  static core::Property SecurityPrivateKeyPassWord;
+
+  EXTENSIONAPI static const core::Property BrokerURL;
+  EXTENSIONAPI static const core::Property ClientID;
+  EXTENSIONAPI static const core::Property UserName;
+  EXTENSIONAPI static const core::Property PassWord;
+  EXTENSIONAPI static const core::Property CleanSession;
+  EXTENSIONAPI static const core::Property KeepLiveInterval;
+  EXTENSIONAPI static const core::Property ConnectionTimeout;
+  EXTENSIONAPI static const core::Property Topic;
+  EXTENSIONAPI static const core::Property QOS;
+  EXTENSIONAPI static const core::Property SecurityProtocol;
+  EXTENSIONAPI static const core::Property SecurityCA;
+  EXTENSIONAPI static const core::Property SecurityCert;
+  EXTENSIONAPI static const core::Property SecurityPrivateKey;
+  EXTENSIONAPI static const core::Property SecurityPrivateKeyPassWord;
+  static auto properties() {
+    return std::array{
+      BrokerURL,
+      ClientID,
+      UserName,
+      PassWord,
+      CleanSession,
+      KeepLiveInterval,
+      ConnectionTimeout,
+      Topic,
+      QOS,
+      SecurityProtocol,
+      SecurityCA,
+      SecurityCert,
+      SecurityPrivateKey,
+      SecurityPrivateKeyPassWord
+    };
+  }
 
  public:
-  /**
-   * Function that's executed when the processor is scheduled.
-   * @param context process context.
-   * @param sessionFactory process session factory that is used when creating
-   * ProcessSession objects.
-   */
   void onSchedule(const std::shared_ptr<core::ProcessContext> &context, const std::shared_ptr<core::ProcessSessionFactory> &factory) override;
 
   // MQTT async callbacks
@@ -116,14 +114,11 @@ class AbstractMQTTProcessor : public core::Processor {
     processor->reconnect();
   }
   bool reconnect();
-  // enqueue receive MQTT message
   virtual bool enqueueReceiveMQTTMsg(MQTTClient_message* /*message*/) {
     return false;
   }
 
  protected:
-  static const std::set<core::Property> getSupportedProperties();
-
   MQTTClient client_;
   MQTTClient_deliveryToken delivered_token_;
   std::string uri_;
@@ -148,8 +143,4 @@ class AbstractMQTTProcessor : public core::Processor {
 };
 
 
-} /* namespace processors */
-} /* namespace minifi */
-} /* namespace nifi */
-} /* namespace apache */
-} /* namespace org */
+}  // namespace org::apache::nifi::minifi::processors
diff --git a/extensions/mqtt/processors/AbstractMQTTProcessorStaticDefinitions.cpp b/extensions/mqtt/processors/AbstractMQTTProcessorStaticDefinitions.cpp
new file mode 100644
index 000000000..3c2bf433d
--- /dev/null
+++ b/extensions/mqtt/processors/AbstractMQTTProcessorStaticDefinitions.cpp
@@ -0,0 +1,65 @@
+/**
+ * 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 "AbstractMQTTProcessor.h"
+#include "ConsumeMQTT.h"
+#include "PublishMQTT.h"
+#include "core/Resource.h"
+
+// FIXME(fgerlits): we need to put all these static definitions in a single file so that they are executed in this order at runtime
+// when https://issues.apache.org/jira/browse/MINIFICPP-1825 is closed, these definitions should be moved back to the cpp file of the class to which they belong
+
+namespace org::apache::nifi::minifi::processors {
+
+// AbstractMQTTProcessor
+
+const core::Property AbstractMQTTProcessor::BrokerURL("Broker URI", "The URI to use to connect to the MQTT broker", "");
+const core::Property AbstractMQTTProcessor::CleanSession("Session state", "Whether to start afresh or resume previous flows. See the allowable value descriptions for more details", "true");
+const core::Property AbstractMQTTProcessor::ClientID("Client ID", "MQTT client ID to use", "");
+const core::Property AbstractMQTTProcessor::UserName("Username", "Username to use when connecting to the broker", "");
+const core::Property AbstractMQTTProcessor::PassWord("Password", "Password to use when connecting to the broker", "");
+const core::Property AbstractMQTTProcessor::KeepLiveInterval("Keep Alive Interval", "Defines the maximum time interval between messages sent or received", "60 sec");
+const core::Property AbstractMQTTProcessor::ConnectionTimeout("Connection Timeout", "Maximum time interval the client will wait for the network connection to the MQTT server", "30 sec");
+const core::Property AbstractMQTTProcessor::QOS("Quality of Service", "The Quality of Service(QoS) to send the message with. Accepts three values '0', '1' and '2'", "MQTT_QOS_0");
+const core::Property AbstractMQTTProcessor::Topic("Topic", "The topic to publish the message to", "");
+const core::Property AbstractMQTTProcessor::SecurityProtocol("Security Protocol", "Protocol used to communicate with brokers", "");
+const core::Property AbstractMQTTProcessor::SecurityCA("Security CA", "File or directory path to CA certificate(s) for verifying the broker's key", "");
+const core::Property AbstractMQTTProcessor::SecurityCert("Security Cert", "Path to client's public key (PEM) used for authentication", "");
+const core::Property AbstractMQTTProcessor::SecurityPrivateKey("Security Private Key", "Path to client's private key (PEM) used for authentication", "");
+const core::Property AbstractMQTTProcessor::SecurityPrivateKeyPassWord("Security Pass Phrase", "Private key passphrase", "");
+
+
+// ConsumeMQTT
+
+const core::Property ConsumeMQTT::MaxFlowSegSize("Max Flow Segment Size", "Maximum flow content payload segment size for the MQTT record", "");
+const core::Property ConsumeMQTT::QueueBufferMaxMessage("Queue Max Message", "Maximum number of messages allowed on the received MQTT queue", "");
+
+const core::Relationship ConsumeMQTT::Success("success", "FlowFiles that are sent successfully to the destination are transferred to this relationship");
+
+REGISTER_RESOURCE(ConsumeMQTT, Processor);
+
+
+// PublishMQTT
+
+const core::Property PublishMQTT::Retain("Retain", "Retain MQTT published record in broker", "false");
+const core::Property PublishMQTT::MaxFlowSegSize("Max Flow Segment Size", "Maximum flow content payload segment size for the MQTT record", "");
+
+const core::Relationship PublishMQTT::Success("success", "FlowFiles that are sent successfully to the destination are transferred to this relationship");
+const core::Relationship PublishMQTT::Failure("failure", "FlowFiles that failed to send to the destination are transferred to this relationship");
+
+REGISTER_RESOURCE(PublishMQTT, Processor);
+
+}  // namespace org::apache::nifi::minifi::processors
diff --git a/extensions/mqtt/processors/ConsumeMQTT.cpp b/extensions/mqtt/processors/ConsumeMQTT.cpp
index 8290c8308..4eccda33b 100644
--- a/extensions/mqtt/processors/ConsumeMQTT.cpp
+++ b/extensions/mqtt/processors/ConsumeMQTT.cpp
@@ -1,7 +1,4 @@
 /**
- * @file ConsumeMQTT.cpp
- * ConsumeMQTT class implementation
- *
  * 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.
@@ -24,8 +21,8 @@
 #include <memory>
 #include <string>
 #include <map>
-#include <set>
 #include <cinttypes>
+#include <vector>
 
 #include "utils/TimeUtil.h"
 #include "utils/StringUtils.h"
@@ -33,25 +30,11 @@
 #include "core/ProcessSession.h"
 #include "core/Resource.h"
 
-namespace org {
-namespace apache {
-namespace nifi {
-namespace minifi {
-namespace processors {
-
-core::Property ConsumeMQTT::MaxFlowSegSize("Max Flow Segment Size", "Maximum flow content payload segment size for the MQTT record", "");
-core::Property ConsumeMQTT::QueueBufferMaxMessage("Queue Max Message", "Maximum number of messages allowed on the received MQTT queue", "");
-
-core::Relationship ConsumeMQTT::Success("success", "FlowFiles that are sent successfully to the destination are transferred to this relationship");
+namespace org::apache::nifi::minifi::processors {
 
 void ConsumeMQTT::initialize() {
-  // Set the supported properties
-  std::set<core::Property> properties(AbstractMQTTProcessor::getSupportedProperties());
-  properties.insert(MaxFlowSegSize);
-  properties.insert(QueueBufferMaxMessage);
-  setSupportedProperties(properties);
-  // Set the supported relationships
-  setSupportedRelationships({Success});
+  setSupportedProperties(properties());
+  setSupportedRelationships(relationships());
 }
 
 bool ConsumeMQTT::enqueueReceiveMQTTMsg(MQTTClient_message *message) {
@@ -121,10 +104,4 @@ void ConsumeMQTT::onTrigger(const std::shared_ptr<core::ProcessContext>& /*conte
   }
 }
 
-REGISTER_RESOURCE(ConsumeMQTT, "This Processor gets the contents of a FlowFile from a MQTT broker for a specified topic. The the payload of the MQTT message becomes content of a FlowFile");
-
-} /* namespace processors */
-} /* namespace minifi */
-} /* namespace nifi */
-} /* namespace apache */
-} /* namespace org */
+}  // namespace org::apache::nifi::minifi::processors
diff --git a/extensions/mqtt/processors/ConsumeMQTT.h b/extensions/mqtt/processors/ConsumeMQTT.h
index cad8461af..c49884d81 100644
--- a/extensions/mqtt/processors/ConsumeMQTT.h
+++ b/extensions/mqtt/processors/ConsumeMQTT.h
@@ -1,7 +1,4 @@
 /**
- * @file ConsumeMQTT.h
- * ConsumeMQTT class declaration
- *
  * 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.
@@ -33,56 +30,53 @@
 #include "concurrentqueue.h"
 #include "MQTTClient.h"
 #include "AbstractMQTTProcessor.h"
+#include "utils/ArrayUtils.h"
 #include "utils/gsl.h"
 
-namespace org {
-namespace apache {
-namespace nifi {
-namespace minifi {
-namespace processors {
+namespace org::apache::nifi::minifi::processors {
 
 #define MQTT_TOPIC_ATTRIBUTE "mqtt.topic"
 #define MQTT_BROKER_ATTRIBUTE "mqtt.broker"
 
-// ConsumeMQTT Class
 class ConsumeMQTT : public processors::AbstractMQTTProcessor {
  public:
-  // Constructor
-  /*!
-   * Create a new processor
-   */
   explicit ConsumeMQTT(const std::string& name, const utils::Identifier& uuid = {})
       : processors::AbstractMQTTProcessor(name, uuid) {
     isSubscriber_ = true;
     maxQueueSize_ = 100;
     maxSegSize_ = ULLONG_MAX;
   }
-  // Destructor
   ~ConsumeMQTT() override {
     MQTTClient_message *message;
     while (queue_.try_dequeue(message)) {
       MQTTClient_freeMessage(&message);
     }
   }
-  // Processor Name
-  static constexpr char const* ProcessorName = "ConsumeMQTT";
-  // Supported Properties
-  static core::Property MaxFlowSegSize;
-  static core::Property QueueBufferMaxMessage;
 
-  static core::Relationship Success;
+  EXTENSIONAPI static constexpr const char* Description = "This Processor gets the contents of a FlowFile from a MQTT broker for a specified topic. "
+      "The the payload of the MQTT message becomes content of a FlowFile";
+
+  EXTENSIONAPI static const core::Property MaxFlowSegSize;
+  EXTENSIONAPI static const core::Property QueueBufferMaxMessage;
+  static auto properties() {
+    return utils::array_cat(AbstractMQTTProcessor::properties(), std::array{
+      MaxFlowSegSize,
+      QueueBufferMaxMessage
+    });
+  }
+
+  EXTENSIONAPI static const core::Relationship Success;
+  static auto relationships() { return std::array{Success}; }
+
+  EXTENSIONAPI static constexpr bool SupportsDynamicProperties = false;
+  EXTENSIONAPI static constexpr bool SupportsDynamicRelationships = false;
+  EXTENSIONAPI static constexpr core::annotation::Input InputRequirement = core::annotation::Input::INPUT_FORBIDDEN;
+  EXTENSIONAPI static constexpr bool IsSingleThreaded = false;
+
+  ADD_COMMON_VIRTUAL_FUNCTIONS_FOR_PROCESSORS
 
- public:
-  /**
-   * Function that's executed when the processor is scheduled.
-   * @param context process context.
-   * @param sessionFactory process session factory that is used when creating
-   * ProcessSession objects.
-   */
   void onSchedule(const std::shared_ptr<core::ProcessContext> &context, const std::shared_ptr<core::ProcessSessionFactory> &factory) override;
-  // OnTrigger method, implemented by NiFi ConsumeMQTT
   void onTrigger(const std::shared_ptr<core::ProcessContext> &context, const std::shared_ptr<core::ProcessSession> &session) override;
-  // Initialize, over write by NiFi ConsumeMQTT
   void initialize() override;
   bool enqueueReceiveMQTTMsg(MQTTClient_message *message) override;
 
@@ -95,10 +89,6 @@ class ConsumeMQTT : public processors::AbstractMQTTProcessor {
   }
 
  private:
-  core::annotation::Input getInputRequirement() const override {
-    return core::annotation::Input::INPUT_FORBIDDEN;
-  }
-
   std::shared_ptr<core::logging::Logger> logger_ = core::logging::LoggerFactory<ConsumeMQTT>::getLogger();
   std::mutex mutex_;
   uint64_t maxQueueSize_;
@@ -106,8 +96,4 @@ class ConsumeMQTT : public processors::AbstractMQTTProcessor {
   moodycamel::ConcurrentQueue<MQTTClient_message *> queue_;
 };
 
-} /* namespace processors */
-} /* namespace minifi */
-} /* namespace nifi */
-} /* namespace apache */
-} /* namespace org */
+}  // namespace org::apache::nifi::minifi::processors
diff --git a/extensions/mqtt/processors/ConvertBase.cpp b/extensions/mqtt/processors/ConvertBase.cpp
index a21d204e7..81658a679 100644
--- a/extensions/mqtt/processors/ConvertBase.cpp
+++ b/extensions/mqtt/processors/ConvertBase.cpp
@@ -1,5 +1,4 @@
 /**
- *
  * 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.
@@ -15,39 +14,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-#include <stdio.h>
-#include <algorithm>
 #include <memory>
 #include <string>
-#include <map>
-#include <set>
-#include "utils/TimeUtil.h"
-#include "utils/StringUtils.h"
 #include "core/ProcessContext.h"
-#include "core/ProcessSession.h"
 #include "ConvertBase.h"
 #include "c2/PayloadSerializer.h"
-#include "utils/ByteArrayCallback.h"
-namespace org {
-namespace apache {
-namespace nifi {
-namespace minifi {
-namespace processors {
 
-core::Property ConvertBase::MQTTControllerService("MQTT Controller Service", "Name of controller service that will be used for MQTT interactivity", "");
-core::Property ConvertBase::ListeningTopic("Listening Topic", "Name of topic to listen to", "");
-core::Relationship ConvertBase::Success("success", "All files are routed to success");
+namespace org::apache::nifi::minifi::processors {
+
+const core::Relationship ConvertBase::Success("success", "All files are routed to success");
 
 void ConvertBase::initialize() {
-  // Set the supported properties
-  std::set<core::Property> properties;
-  properties.insert(MQTTControllerService);
-  properties.insert(ListeningTopic);
-  setSupportedProperties(properties);
-  // Set the supported relationships
-  std::set<core::Relationship> relationships;
-  relationships.insert(Success);
-  setSupportedRelationships(relationships);
+  setSupportedProperties(properties());
+  setSupportedRelationships(relationships());
 }
 
 void ConvertBase::onSchedule(const std::shared_ptr<core::ProcessContext> &context, const std::shared_ptr<core::ProcessSessionFactory>& /*sessionFactory*/) {
@@ -62,8 +41,4 @@ void ConvertBase::onSchedule(const std::shared_ptr<core::ProcessContext> &contex
   }
 }
 
-} /* namespace processors */
-} /* namespace minifi */
-} /* namespace nifi */
-} /* namespace apache */
-} /* namespace org */
+}  // namespace org::apache::nifi::minifi::processors
diff --git a/extensions/mqtt/processors/ConvertBase.h b/extensions/mqtt/processors/ConvertBase.h
index d994bfa40..392dfe831 100644
--- a/extensions/mqtt/processors/ConvertBase.h
+++ b/extensions/mqtt/processors/ConvertBase.h
@@ -1,5 +1,4 @@
 /**
- *
  * 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.
@@ -30,11 +29,7 @@
 #include "MQTTClient.h"
 #include "c2/protocols/RESTProtocol.h"
 
-namespace org {
-namespace apache {
-namespace nifi {
-namespace minifi {
-namespace processors {
+namespace org::apache::nifi::minifi::processors {
 
 /**
  * Purpose: Provides base functionality for mqtt conversion classes.
@@ -42,45 +37,31 @@ namespace processors {
  */
 class ConvertBase : public core::Processor, public minifi::c2::RESTProtocol {
  public:
-  // Constructor
-  /*!
-   * Create a new processor
-   */
   explicit ConvertBase(const std::string& name, const utils::Identifier& uuid = {})
       : core::Processor(name, uuid) {
   }
-  // Destructor
   virtual ~ConvertBase() = default;
-  // Supported Properties
-  static core::Property MQTTControllerService;
-  static core::Property ListeningTopic;
 
-  static core::Relationship Success;
+  EXTENSIONAPI static const core::Property MQTTControllerService;
+  EXTENSIONAPI static const core::Property ListeningTopic;
+  static auto properties() {
+    return std::array{
+      MQTTControllerService,
+      ListeningTopic
+    };
+  }
+
+  static const core::Relationship Success;
+  static auto relationships() { return std::array{Success}; }
 
  public:
-  /**
-   * Initialization of the processor
-   */
   void initialize() override;
-  /**
-   * Function that's executed when the processor is scheduled.
-   * @param context process context.
-   * @param sessionFactory process session factory that is used when creating
-   * ProcessSession objects.
-   */
   void onSchedule(const std::shared_ptr<core::ProcessContext> &context, const std::shared_ptr<core::ProcessSessionFactory> &sessionFactory) override;
 
  protected:
-  /**
-   * MQTT controller service.
-   */
   std::shared_ptr<controllers::MQTTControllerService> mqtt_service_;
 
   std::string listening_topic;
 };
 
-} /* namespace processors */
-} /* namespace minifi */
-} /* namespace nifi */
-} /* namespace apache */
-} /* namespace org */
+}  // namespace org::apache::nifi::minifi::processors
diff --git a/extensions/mqtt/processors/ConvertBaseStaticDefinitions.cpp b/extensions/mqtt/processors/ConvertBaseStaticDefinitions.cpp
new file mode 100644
index 000000000..a31b0a1c2
--- /dev/null
+++ b/extensions/mqtt/processors/ConvertBaseStaticDefinitions.cpp
@@ -0,0 +1,51 @@
+/**
+ * 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 "ConvertBase.h"
+#include "ConvertHeartBeat.h"
+#include "ConvertJSONAck.h"
+#include "ConvertUpdate.h"
+#include "core/Resource.h"
+
+// FIXME(fgerlits): we need to put all these static definitions in a single file so that they are executed in this order at runtime
+// when https://issues.apache.org/jira/browse/MINIFICPP-1825 is closed, these definitions should be moved back to the cpp file of the class to which they belong
+
+namespace org::apache::nifi::minifi::processors {
+
+// ConvertBase
+
+const core::Property ConvertBase::MQTTControllerService("MQTT Controller Service", "Name of controller service that will be used for MQTT interactivity", "");
+const core::Property ConvertBase::ListeningTopic("Listening Topic", "Name of topic to listen to", "");
+
+
+// ConvertHeartBeat
+
+REGISTER_RESOURCE(ConvertHeartBeat, InternalResource);
+
+
+// ConvertJSONAck
+
+REGISTER_RESOURCE(ConvertJSONAck, InternalResource);
+
+
+// ConvertUpdate
+
+core::Property ConvertUpdate::SSLContext("SSL Context Service", "The SSL Context Service used to provide client certificate information for TLS/SSL (https) connections.", "");
+
+REGISTER_RESOURCE(ConvertUpdate, InternalResource);
+
+}  // namespace org::apache::nifi::minifi::processors
diff --git a/extensions/mqtt/processors/ConvertHeartBeat.cpp b/extensions/mqtt/processors/ConvertHeartBeat.cpp
index 69446b5de..647602073 100644
--- a/extensions/mqtt/processors/ConvertHeartBeat.cpp
+++ b/extensions/mqtt/processors/ConvertHeartBeat.cpp
@@ -1,5 +1,4 @@
 /**
- *
  * 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.
@@ -15,15 +14,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-#include <stdio.h>
-#include <algorithm>
 #include <memory>
-#include <string>
-#include <map>
 #include <vector>
-#include <set>
 
-#include "utils/TimeUtil.h"
 #include "utils/StringUtils.h"
 #include "core/ProcessContext.h"
 #include "core/ProcessSession.h"
@@ -31,6 +24,7 @@
 #include "ConvertHeartBeat.h"
 #include "c2/PayloadSerializer.h"
 #include "utils/ByteArrayCallback.h"
+
 namespace org {
 namespace apache {
 namespace nifi {
@@ -70,8 +64,6 @@ void ConvertHeartBeat::onTrigger(const std::shared_ptr<core::ProcessContext> &co
   }
 }
 
-REGISTER_INTERNAL_RESOURCE(ConvertHeartBeat);
-
 } /* namespace processors */
 } /* namespace minifi */
 } /* namespace nifi */
diff --git a/extensions/mqtt/processors/ConvertHeartBeat.h b/extensions/mqtt/processors/ConvertHeartBeat.h
index 241fce197..980f05adf 100644
--- a/extensions/mqtt/processors/ConvertHeartBeat.h
+++ b/extensions/mqtt/processors/ConvertHeartBeat.h
@@ -1,5 +1,4 @@
 /**
- *
  * 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.
@@ -34,27 +33,19 @@ namespace processors {
 /*
  * Purpose: ConvertHeartBeat converts heatbeats into MQTT messages.
  */
-class ConvertHeartBeat: public ConvertBase{
+class ConvertHeartBeat: public ConvertBase {
  public:
-  // Constructor
-  /*!
-   * Create a new processor
-   */
   explicit ConvertHeartBeat(const std::string& name, const utils::Identifier& uuid = {})
     : ConvertBase(name, uuid) {
   }
-  // Destructor
   ~ConvertHeartBeat() override = default;
-  // Processor Name
-  static constexpr char const* ProcessorName = "ConvertHeartBeat";
 
- public:
-  /**
-   * Function that's executed when the processor is triggered.
-   * @param context process context.
-   * @param sessionFactory process session factory that is used when creating
-   * ProcessSession objects.
-   */
+  EXTENSIONAPI static constexpr bool SupportsDynamicProperties = false;
+  EXTENSIONAPI static constexpr bool SupportsDynamicRelationships = false;
+  EXTENSIONAPI static constexpr core::annotation::Input InputRequirement = core::annotation::Input::INPUT_ALLOWED;
+  EXTENSIONAPI static constexpr bool IsSingleThreaded = false;
+
+  ADD_COMMON_VIRTUAL_FUNCTIONS_FOR_PROCESSORS
 
   void onTrigger(const std::shared_ptr<core::ProcessContext> &context, const std::shared_ptr<core::ProcessSession> &session) override;
 
diff --git a/extensions/mqtt/processors/ConvertJSONAck.cpp b/extensions/mqtt/processors/ConvertJSONAck.cpp
index 1d44905ec..10d47c897 100644
--- a/extensions/mqtt/processors/ConvertJSONAck.cpp
+++ b/extensions/mqtt/processors/ConvertJSONAck.cpp
@@ -1,5 +1,4 @@
 /**
- *
  * 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.
@@ -42,6 +41,7 @@ std::string ConvertJSONAck::parseTopicName(const std::string &json) {
   }
   return topic;
 }
+
 void ConvertJSONAck::onTrigger(const std::shared_ptr<core::ProcessContext> &context, const std::shared_ptr<core::ProcessSession> &session) {
   if (nullptr == mqtt_service_) {
     context->yield();
@@ -81,6 +81,4 @@ void ConvertJSONAck::onTrigger(const std::shared_ptr<core::ProcessContext> &cont
   session->transfer(flow, Success);
 }
 
-REGISTER_INTERNAL_RESOURCE(ConvertJSONAck);
-
 }  // namespace org::apache::nifi::minifi::processors
diff --git a/extensions/mqtt/processors/ConvertJSONAck.h b/extensions/mqtt/processors/ConvertJSONAck.h
index bfe458b56..9496e1461 100644
--- a/extensions/mqtt/processors/ConvertJSONAck.h
+++ b/extensions/mqtt/processors/ConvertJSONAck.h
@@ -1,7 +1,4 @@
 /**
- * @file ConvertAck.h
- * ConvertAck class declaration
- *
  * 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.
@@ -29,35 +26,22 @@
 #include "core/logging/LoggerConfiguration.h"
 #include "ConvertBase.h"
 
-
 namespace org::apache::nifi::minifi::processors {
 
-/**
- * Purpose: Converts JSON acks into an MQTT consumable by
- * MQTTC2Protocol.
- */
 class ConvertJSONAck : public ConvertBase {
  public:
-  // Constructor
-  /*!
-   * Create a new processor
-   */
   explicit ConvertJSONAck(const std::string& name, const utils::Identifier& uuid = {})
       : ConvertBase(name, uuid) {
   }
-  // Destructor
   ~ConvertJSONAck() override = default;
-  // Processor Name
-  static constexpr char const* ProcessorName = "ConvertJSONAck";
 
+  EXTENSIONAPI static constexpr bool SupportsDynamicProperties = false;
+  EXTENSIONAPI static constexpr bool SupportsDynamicRelationships = false;
+  EXTENSIONAPI static constexpr core::annotation::Input InputRequirement = core::annotation::Input::INPUT_ALLOWED;
+  EXTENSIONAPI static constexpr bool IsSingleThreaded = false;
+
+  ADD_COMMON_VIRTUAL_FUNCTIONS_FOR_PROCESSORS
 
- public:
-  /**
-   * Function that's executed when the processor is scheduled.
-   * @param context process context.
-   * @param sessionFactory process session factory that is used when creating
-   * ProcessSession objects.
-   */
   void onTrigger(const std::shared_ptr<core::ProcessContext> &context, const std::shared_ptr<core::ProcessSession> &session) override;
 
  protected:
diff --git a/extensions/mqtt/processors/ConvertUpdate.cpp b/extensions/mqtt/processors/ConvertUpdate.cpp
index 687b176bf..c604595f1 100644
--- a/extensions/mqtt/processors/ConvertUpdate.cpp
+++ b/extensions/mqtt/processors/ConvertUpdate.cpp
@@ -1,5 +1,4 @@
 /**
- *
  * 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.
@@ -19,7 +18,6 @@
 #include <memory>
 #include <vector>
 #include <algorithm>
-#include <set>
 
 #include "ConvertUpdate.h"
 #include "utils/HTTPClient.h"
@@ -27,13 +25,7 @@
 #include "io/BufferStream.h"
 #include "core/Resource.h"
 
-namespace org {
-namespace apache {
-namespace nifi {
-namespace minifi {
-namespace processors {
-
-core::Property ConvertUpdate::SSLContext("SSL Context Service", "The SSL Context Service used to provide client certificate information for TLS/SSL (https) connections.", "");
+namespace org::apache::nifi::minifi::processors {
 
 void ConvertUpdate::onTrigger(const std::shared_ptr<core::ProcessContext> &context, const std::shared_ptr<core::ProcessSession>& /*session*/) {
   if (nullptr == mqtt_service_) {
@@ -92,22 +84,8 @@ void ConvertUpdate::onTrigger(const std::shared_ptr<core::ProcessContext> &conte
 }
 
 void ConvertUpdate::initialize() {
-  // Set the supported properties
-  std::set<core::Property> properties;
-  properties.insert(MQTTControllerService);
-  properties.insert(ListeningTopic);
-  properties.insert(SSLContext);
-  setSupportedProperties(properties);
-  // Set the supported relationships
-  std::set<core::Relationship> relationships;
-  relationships.insert(Success);
-  setSupportedRelationships(relationships);
+  setSupportedProperties(properties());
+  setSupportedRelationships(relationships());
 }
 
-REGISTER_INTERNAL_RESOURCE(ConvertUpdate);
-
-} /* namespace processors */
-} /* namespace minifi */
-} /* namespace nifi */
-} /* namespace apache */
-} /* namespace org */
+}  // namespace org::apache::nifi::minifi::processors
diff --git a/extensions/mqtt/processors/ConvertUpdate.h b/extensions/mqtt/processors/ConvertUpdate.h
index 9561fb3d0..8a50e1b6e 100644
--- a/extensions/mqtt/processors/ConvertUpdate.h
+++ b/extensions/mqtt/processors/ConvertUpdate.h
@@ -1,5 +1,4 @@
 /**
- *
  * 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.
@@ -26,12 +25,9 @@
 #include "core/Property.h"
 #include "core/logging/LoggerConfiguration.h"
 #include "ConvertBase.h"
+#include "utils/ArrayUtils.h"
 
-namespace org {
-namespace apache {
-namespace nifi {
-namespace minifi {
-namespace processors {
+namespace org::apache::nifi::minifi::processors {
 
 /**
  * Purpose: Converts update messages into the appropriate Restful call
@@ -42,32 +38,22 @@ namespace processors {
  */
 class ConvertUpdate : public ConvertBase {
  public:
-  // Constructor
-  /*!
-   * Create a new processor
-   */
   explicit ConvertUpdate(const std::string& name, const utils::Identifier& uuid = {})
     : ConvertBase(name, uuid) {
   }
-  // Destructor
   ~ConvertUpdate() override = default;
 
-  static core::Property SSLContext;
-  // Processor Name
-  static constexpr char const* ProcessorName = "ConvertUpdate";
+  EXTENSIONAPI static core::Property SSLContext;
+  static auto properties() { return utils::array_cat(ConvertBase::properties(), std::array{SSLContext}); }
 
- public:
-  /**
-     * Initialization of the processor
-     */
-    void initialize() override;
-  /**
-   * Function that's executed when the processor is triggered.
-   * @param context process context.
-   * @param sessionFactory process session factory that is used when creating
-   * ProcessSession objects.
-   */
+  EXTENSIONAPI static constexpr bool SupportsDynamicProperties = false;
+  EXTENSIONAPI static constexpr bool SupportsDynamicRelationships = false;
+  EXTENSIONAPI static constexpr core::annotation::Input InputRequirement = core::annotation::Input::INPUT_ALLOWED;
+  EXTENSIONAPI static constexpr bool IsSingleThreaded = false;
+
+  ADD_COMMON_VIRTUAL_FUNCTIONS_FOR_PROCESSORS
 
+  void initialize() override;
   void onTrigger(const std::shared_ptr<core::ProcessContext> &context, const std::shared_ptr<core::ProcessSession> &session) override;
 
  protected:
@@ -77,8 +63,4 @@ class ConvertUpdate : public ConvertBase {
   std::shared_ptr<core::logging::Logger> logger_ = core::logging::LoggerFactory<ConvertUpdate>::getLogger();
 };
 
-} /* namespace processors */
-} /* namespace minifi */
-} /* namespace nifi */
-} /* namespace apache */
-} /* namespace org */
+}  // namespace org::apache::nifi::minifi::processors
diff --git a/extensions/mqtt/processors/PublishMQTT.cpp b/extensions/mqtt/processors/PublishMQTT.cpp
index 54cfd0aed..bf0afbe6b 100644
--- a/extensions/mqtt/processors/PublishMQTT.cpp
+++ b/extensions/mqtt/processors/PublishMQTT.cpp
@@ -1,7 +1,4 @@
 /**
- * @file PublishMQTT.cpp
- * PublishMQTT class implementation
- *
  * 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.
@@ -25,8 +22,8 @@
 #include <map>
 #include <memory>
 #include <optional>
-#include <set>
 #include <string>
+#include <vector>
 
 #include "utils/TimeUtil.h"
 #include "utils/StringUtils.h"
@@ -34,26 +31,11 @@
 #include "core/ProcessSession.h"
 #include "core/Resource.h"
 
-namespace org {
-namespace apache {
-namespace nifi {
-namespace minifi {
-namespace processors {
-
-core::Property PublishMQTT::Retain("Retain", "Retain MQTT published record in broker", "false");
-core::Property PublishMQTT::MaxFlowSegSize("Max Flow Segment Size", "Maximum flow content payload segment size for the MQTT record", "");
-
-core::Relationship PublishMQTT::Success("success", "FlowFiles that are sent successfully to the destination are transferred to this relationship");
-core::Relationship PublishMQTT::Failure("failure", "FlowFiles that failed to send to the destination are transferred to this relationship");
+namespace org::apache::nifi::minifi::processors {
 
 void PublishMQTT::initialize() {
-  // Set the supported properties
-  std::set<core::Property> properties(AbstractMQTTProcessor::getSupportedProperties());
-  properties.insert(Retain);
-  properties.insert(MaxFlowSegSize);
-  setSupportedProperties(properties);
-  // Set the supported relationships
-  setSupportedRelationships({Success, Failure});
+  setSupportedProperties(properties());
+  setSupportedRelationships(relationships());
 }
 
 void PublishMQTT::onSchedule(const std::shared_ptr<core::ProcessContext> &context, const std::shared_ptr<core::ProcessSessionFactory> &factory) {
@@ -100,10 +82,4 @@ void PublishMQTT::onTrigger(const std::shared_ptr<core::ProcessContext>& /*conte
   }
 }
 
-REGISTER_RESOURCE(PublishMQTT, "PublishMQTT serializes FlowFile content as an MQTT payload, sending the message to the configured topic and broker.");
-
-} /* namespace processors */
-} /* namespace minifi */
-} /* namespace nifi */
-} /* namespace apache */
-} /* namespace org */
+}  // namespace org::apache::nifi::minifi::processors
diff --git a/extensions/mqtt/processors/PublishMQTT.h b/extensions/mqtt/processors/PublishMQTT.h
index ff58f61a0..0b0e2711b 100644
--- a/extensions/mqtt/processors/PublishMQTT.h
+++ b/extensions/mqtt/processors/PublishMQTT.h
@@ -1,7 +1,4 @@
 /**
- * @file PublishMQTT.h
- * PublishMQTT class declaration
- *
  * 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.
@@ -33,38 +30,42 @@
 #include "core/logging/LoggerConfiguration.h"
 #include "MQTTClient.h"
 #include "AbstractMQTTProcessor.h"
+#include "utils/ArrayUtils.h"
 #include "utils/gsl.h"
 
-namespace org {
-namespace apache {
-namespace nifi {
-namespace minifi {
-namespace processors {
+namespace org::apache::nifi::minifi::processors {
 
-// PublishMQTT Class
 class PublishMQTT : public processors::AbstractMQTTProcessor {
  public:
-  // Constructor
-  /*!
-   * Create a new processor
-   */
   explicit PublishMQTT(const std::string& name, const utils::Identifier& uuid = {})
       : processors::AbstractMQTTProcessor(name, uuid) {
     retain_ = false;
     max_seg_size_ = ULLONG_MAX;
   }
-  // Destructor
   ~PublishMQTT() override = default;
-  // Processor Name
-  static constexpr char const* ProcessorName = "PublishMQTT";
-  // Supported Properties
-  static core::Property Retain;
-  static core::Property MaxFlowSegSize;
 
-  static core::Relationship Failure;
-  static core::Relationship Success;
+  EXTENSIONAPI static constexpr const char* Description = "PublishMQTT serializes FlowFile content as an MQTT payload, sending the message to the configured topic and broker.";
+
+  EXTENSIONAPI static const core::Property Retain;
+  EXTENSIONAPI static const core::Property MaxFlowSegSize;
+  static auto properties() {
+    return utils::array_cat(AbstractMQTTProcessor::properties(), std::array{
+      Retain,
+      MaxFlowSegSize
+    });
+  }
+
+  EXTENSIONAPI static const core::Relationship Success;
+  EXTENSIONAPI static const core::Relationship Failure;
+  static auto relationships() { return std::array{Success, Failure}; }
+
+  EXTENSIONAPI static constexpr bool SupportsDynamicProperties = false;
+  EXTENSIONAPI static constexpr bool SupportsDynamicRelationships = false;
+  EXTENSIONAPI static constexpr core::annotation::Input InputRequirement = core::annotation::Input::INPUT_REQUIRED;
+  EXTENSIONAPI static constexpr bool IsSingleThreaded = false;
+
+  ADD_COMMON_VIRTUAL_FUNCTIONS_FOR_PROCESSORS
 
-  // Nest Callback Class for read stream
   class ReadCallback {
    public:
     ReadCallback(uint64_t flow_size, uint64_t max_seg_size, const std::string &key, MQTTClient client, int qos, bool retain, MQTTClient_deliveryToken &token)
@@ -122,30 +123,14 @@ class PublishMQTT : public processors::AbstractMQTTProcessor {
   };
 
  public:
-  /**
-   * Function that's executed when the processor is scheduled.
-   * @param context process context.
-   * @param sessionFactory process session factory that is used when creating
-   * ProcessSession objects.
-   */
   void onSchedule(const std::shared_ptr<core::ProcessContext> &context, const std::shared_ptr<core::ProcessSessionFactory> &factory) override;
-  // OnTrigger method, implemented by NiFi PublishMQTT
   void onTrigger(const std::shared_ptr<core::ProcessContext> &context, const std::shared_ptr<core::ProcessSession> &session) override;
-  // Initialize, over write by NiFi PublishMQTT
   void initialize() override;
 
  private:
-  core::annotation::Input getInputRequirement() const override {
-    return core::annotation::Input::INPUT_REQUIRED;
-  }
-
   uint64_t max_seg_size_;
   bool retain_;
   std::shared_ptr<core::logging::Logger> logger_ = core::logging::LoggerFactory<PublishMQTT>::getLogger();
 };
 
-} /* namespace processors */
-} /* namespace minifi */
-} /* namespace nifi */
-} /* namespace apache */
-} /* namespace org */
+}  // namespace org::apache::nifi::minifi::processors
diff --git a/extensions/mqtt/protocol/MQTTC2Protocol.cpp b/extensions/mqtt/protocol/MQTTC2Protocol.cpp
index a9b104569..565519e57 100644
--- a/extensions/mqtt/protocol/MQTTC2Protocol.cpp
+++ b/extensions/mqtt/protocol/MQTTC2Protocol.cpp
@@ -1,5 +1,4 @@
 /**
- *
  * 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.
@@ -19,11 +18,7 @@
 #include "core/Resource.h"
 #include "properties/Configuration.h"
 
-namespace org {
-namespace apache {
-namespace nifi {
-namespace minifi {
-namespace c2 {
+namespace org::apache::nifi::minifi::c2 {
 
 MQTTC2Protocol::MQTTC2Protocol(const std::string& name, const utils::Identifier& uuid)
     : C2Protocol(name, uuid) {
@@ -97,10 +92,6 @@ C2Payload MQTTC2Protocol::serialize(const C2Payload &payload) {
   return C2Payload(payload.getOperation(), state::UpdateState::READ_ERROR);
 }
 
-REGISTER_INTERNAL_RESOURCE(MQTTC2Protocol);
+REGISTER_RESOURCE(MQTTC2Protocol, InternalResource);
 
-} /* namespace c2 */
-} /* namespace minifi */
-} /* namespace nifi */
-} /* namespace apache */
-} /* namespace org */
+}  // namespace org::apache::nifi::minifi::c2
diff --git a/extensions/mqtt/protocol/MQTTC2Protocol.h b/extensions/mqtt/protocol/MQTTC2Protocol.h
index f4ad9b132..096f98275 100644
--- a/extensions/mqtt/protocol/MQTTC2Protocol.h
+++ b/extensions/mqtt/protocol/MQTTC2Protocol.h
@@ -1,5 +1,4 @@
 /**
- *
  * 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.
@@ -31,11 +30,7 @@
 #include "agent/agent_version.h"
 #include "c2/PayloadSerializer.h"
 
-namespace org {
-namespace apache {
-namespace nifi {
-namespace minifi {
-namespace c2 {
+namespace org::apache::nifi::minifi::c2 {
 
 /**
  * Purpose: Implementation of the MQTT C2 protocol. Serializes messages to and from
@@ -45,14 +40,12 @@ class MQTTC2Protocol : public C2Protocol {
  public:
   explicit MQTTC2Protocol(const std::string& name, const utils::Identifier& uuid = {});
 
-  virtual ~MQTTC2Protocol();
+  ~MQTTC2Protocol() override;
+
+  static auto properties() { return std::array<core::Property, 0>{}; }
+  EXTENSIONAPI static constexpr bool SupportsDynamicProperties = false;
+  EXTENSIONAPI static constexpr bool SupportsDynamicRelationships = false;
 
-  /**
-   * Consume the payload.
-   * @param url to evaluate.
-   * @param payload payload to consume.
-   * @direction direction of operation.
-   */
   C2Payload consumePayload(const std::string &url, const C2Payload &payload, Direction direction, bool async) override;
 
   C2Payload consumePayload(const C2Payload &payload, Direction /*direction*/, bool /*async*/) override {
@@ -85,8 +78,4 @@ class MQTTC2Protocol : public C2Protocol {
   std::string controller_service_name_;
 };
 
-} /* namespace c2 */
-} /* namespace minifi */
-} /* namespace nifi */
-} /* namespace apache */
-} /* namespace org */
+}  // namespace org::apache::nifi::minifi::c2
diff --git a/extensions/opc/include/fetchopc.h b/extensions/opc/include/fetchopc.h
index d430ec7e9..4be75f717 100644
--- a/extensions/opc/include/fetchopc.h
+++ b/extensions/opc/include/fetchopc.h
@@ -1,6 +1,4 @@
 /**
- * FetchOPC class declaration
- *
  * 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.
@@ -31,38 +29,49 @@
 #include "core/Property.h"
 #include "controllers/SSLContextService.h"
 #include "core/logging/LoggerConfiguration.h"
+#include "utils/ArrayUtils.h"
 #include "utils/Id.h"
 #include "utils/gsl.h"
 
-namespace org {
-namespace apache {
-namespace nifi {
-namespace minifi {
-namespace processors {
+namespace org::apache::nifi::minifi::processors {
 
 class FetchOPCProcessor : public BaseOPCProcessor {
  public:
-  static constexpr char const* ProcessorName = "FetchOPC";
-  // Supported Properties
-  static core::Property NodeIDType;
-  static core::Property NodeID;
-  static core::Property NameSpaceIndex;
-  static core::Property MaxDepth;
-  static core::Property Lazy;
-
-  // Supported Relationships
-  static core::Relationship Success;
-  static core::Relationship Failure;
-
   explicit FetchOPCProcessor(const std::string& name, const utils::Identifier& uuid = {})
       : BaseOPCProcessor(name, uuid), nameSpaceIdx_(0), nodesFound_(0), variablesFound_(0), maxDepth_(0) {
     logger_ = core::logging::LoggerFactory<FetchOPCProcessor>::getLogger();
   }
 
-  void onSchedule(const std::shared_ptr<core::ProcessContext> &context, const std::shared_ptr<core::ProcessSessionFactory> &factory) override;
+  EXTENSIONAPI static constexpr const char* Description = "Fetches OPC-UA node";
 
-  void onTrigger(const std::shared_ptr<core::ProcessContext> &context, const std::shared_ptr<core::ProcessSession> &session) override;
+  EXTENSIONAPI static const core::Property NodeIDType;
+  EXTENSIONAPI static const core::Property NodeID;
+  EXTENSIONAPI static const core::Property NameSpaceIndex;
+  EXTENSIONAPI static const core::Property MaxDepth;
+  EXTENSIONAPI static const core::Property Lazy;
+  static auto properties() {
+    return utils::array_cat(BaseOPCProcessor::properties(), std::array{
+      NodeIDType,
+      NodeID,
+      NameSpaceIndex,
+      MaxDepth,
+      Lazy
+    });
+  }
+
+  EXTENSIONAPI static const core::Relationship Success;
+  EXTENSIONAPI static const core::Relationship Failure;
+  static auto relationships() { return std::array{Success, Failure}; }
 
+  EXTENSIONAPI static constexpr bool SupportsDynamicProperties = false;
+  EXTENSIONAPI static constexpr bool SupportsDynamicRelationships = false;
+  EXTENSIONAPI static constexpr core::annotation::Input InputRequirement = core::annotation::Input::INPUT_FORBIDDEN;
+  EXTENSIONAPI static constexpr bool IsSingleThreaded = true;
+
+  ADD_COMMON_VIRTUAL_FUNCTIONS_FOR_PROCESSORS
+
+  void onSchedule(const std::shared_ptr<core::ProcessContext> &context, const std::shared_ptr<core::ProcessSessionFactory> &factory) override;
+  void onTrigger(const std::shared_ptr<core::ProcessContext> &context, const std::shared_ptr<core::ProcessSession> &session) override;
   void initialize() override;
 
  protected:
@@ -80,16 +89,8 @@ class FetchOPCProcessor : public BaseOPCProcessor {
   bool lazy_mode_;
 
  private:
-  core::annotation::Input getInputRequirement() const override {
-    return core::annotation::Input::INPUT_FORBIDDEN;
-  }
-
   std::vector<UA_NodeId> translatedNodeIDs_;  // Only used when user provides path, path->nodeid translation is only done once
   std::unordered_map<std::string, std::string> node_timestamp_;  // Key = Full path, Value = Timestamp
 };
 
-} /* namespace processors */
-} /* namespace minifi */
-} /* namespace nifi */
-} /* namespace apache */
-} /* namespace org */
+}  // namespace org::apache::nifi::minifi::processors
diff --git a/extensions/opc/include/opc.h b/extensions/opc/include/opc.h
index 64728468f..e6bdb4b79 100644
--- a/extensions/opc/include/opc.h
+++ b/extensions/opc/include/opc.h
@@ -21,6 +21,7 @@
 #include <string>
 #include <functional>
 #include <map>
+#include <set>
 #include <vector>
 #include <memory>
 
@@ -125,6 +126,8 @@ static std::map<std::string, OPCNodeDataType>  StringToOPCDataTypeMap = {{"Int64
                                                                          {"UInt32", OPCNodeDataType::UInt32}, {"Boolean", OPCNodeDataType::Boolean}, {"Float", OPCNodeDataType::Float},
                                                                          {"Double", OPCNodeDataType::Double}, {"String", OPCNodeDataType::String}};
 
+std::set<std::string> stringToOPCDataTypeMapKeys();
+
 std::string nodeValue2String(const NodeData& nd);
 
 std::string OPCDateTime2String(UA_DateTime raw_date);
diff --git a/extensions/opc/include/opcbase.h b/extensions/opc/include/opcbase.h
index 81d25e5c1..3848ca153 100644
--- a/extensions/opc/include/opcbase.h
+++ b/extensions/opc/include/opcbase.h
@@ -1,6 +1,4 @@
 /**
- * OPCBase class declaration
... 16045 lines suppressed ...