You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by bb...@apache.org on 2019/11/08 17:33:39 UTC

[nifi] branch master updated: NIFI-6707: Addition of nifi-system-tests module. Updates to EmbeddedNiFi, NiFi and SystemBundle to properly support classloader isolation that is needed for creating system-level tests and isolating the classpath of the tests themselves from the NiFi classpath. Fixed bootstrap listener to ensure that it always closes socket before calling shutdown(). Failure to do so can result in RunNiFi sometimes timing out while waiting for a response from NiFi

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

bbende pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/nifi.git


The following commit(s) were added to refs/heads/master by this push:
     new 5f4ce8f  NIFI-6707: Addition of nifi-system-tests module. Updates to EmbeddedNiFi, NiFi and SystemBundle to properly support classloader isolation that is needed for creating system-level tests and isolating the classpath of the tests themselves from the NiFi classpath.  Fixed bootstrap listener to ensure that it always closes socket before calling shutdown(). Failure to do so can result in RunNiFi sometimes timing out while waiting for a response from NiFi
5f4ce8f is described below

commit 5f4ce8f431d1cb90451102efb516140d87610c52
Author: Mark Payne <ma...@hotmail.com>
AuthorDate: Tue Sep 24 14:34:39 2019 -0400

    NIFI-6707: Addition of nifi-system-tests module. Updates to EmbeddedNiFi, NiFi and SystemBundle to properly support classloader isolation that is needed for creating system-level tests and isolating the classpath of the tests themselves from the NiFi classpath.  Fixed bootstrap listener to ensure that it always closes socket before calling shutdown(). Failure to do so can result in RunNiFi sometimes timing out while waiting for a response from NiFi
    
    NIFI-6707: Added System Test to verify behavior when services depend on one another
    
    NIFI-6707: Ensure that when tearing down flow after test, we wait for all processors and controller services to complete stop/disable before attempting to delete them
    
    NIFI-6707: Fixed bug in RemoteProcessGroupIT that caused the test to fail intermittently based on timing. Improved logging for Process Group, Standard Process Group
    
    NIFI-6707: Updated to include java11 subdirectory under lib/
    
    NIFI-6707: Rebased against master and changed dependencies from 1.10.0-SNAPSHOT to 1.11.0-SNAPSHOT
    
    NIFI-6707: Updates to see if it will help travis build correctly
    
    NIFI-6707: Commenting out java commands in bootstrap.conf files
    
    This closes #3831.
    
    Signed-off-by: Bryan Bende <bb...@apache.org>
---
 .travis.yml                                        |  10 +-
 .../java/org/apache/nifi/bootstrap/RunNiFi.java    | 145 ++---
 .../java/org/apache/nifi/util/file/FileUtils.java  |   6 +-
 .../state/server/ZooKeeperStateServer.java         |  30 +-
 .../apache/nifi/groups/StandardProcessGroup.java   |  13 +-
 .../nifi/remote/StandardRemoteProcessGroup.java    | 105 +++-
 .../nifi/controller/StandardFlowServiceSpec.groovy | 130 ----
 .../nifi/integration/versioned/ImportFlowIT.java   |   4 +-
 .../java/org/apache/nifi/nar/SystemBundle.java     |   8 +-
 .../java/org/apache/nifi/BootstrapListener.java    |   1 +
 .../main/java/org/apache/nifi/EmbeddedNiFi.java    |   7 +
 .../src/main/java/org/apache/nifi/NiFi.java        |   9 +-
 .../nifi/remote/StandardRemoteGroupPort.java       |  10 +-
 .../nifi-system-test-extensions-nar/pom.xml        |  43 ++
 .../nifi-system-test-extensions/pom.xml            |  41 ++
 .../apache/nifi/cs/tests/system/SleepService.java  |  18 +-
 .../nifi/cs/tests/system/StandardSleepService.java | 118 ++++
 .../nifi/processors/tests/system/CountEvents.java  |  97 +++
 .../processors/tests/system/GenerateFlowFile.java  | 136 +++++
 .../apache/nifi/processors/tests/system/Sleep.java | 141 +++++
 .../tests/system/ValidateFileExists.java           |  59 ++
 .../org.apache.nifi.controller.ControllerService   |  16 +
 .../services/org.apache.nifi.processor.Processor   |  19 +
 .../nifi-system-test-extensions-bundle/pom.xml     |  35 ++
 nifi-system-tests/nifi-system-test-suite/pom.xml   | 207 +++++++
 .../src/test/assembly/dependencies.xml             |  80 +++
 .../nifi/tests/system/AggregateNiFiInstance.java   | 128 ++++
 .../nifi/tests/system/InstanceConfiguration.java   | 128 ++++
 .../apache/nifi/tests/system/NiFiClientUtil.java   | 662 +++++++++++++++++++++
 .../org/apache/nifi/tests/system/NiFiInstance.java | 103 ++++
 .../nifi/tests/system/NiFiInstanceFactory.java     |  16 +-
 .../org/apache/nifi/tests/system/NiFiSystemIT.java | 238 ++++++++
 .../system/SpawnedClusterNiFiInstanceFactory.java  |  66 ++
 .../SpawnedStandaloneNiFiInstanceFactory.java      | 293 +++++++++
 .../clustering/JoinClusterAdjustStateIT.java       | 105 ++++
 .../clustering/NodeRestartWithNewNodeIdIT.java     | 102 ++++
 .../clustering/RestartWithDifferentPort.java       |  79 +++
 .../system/clustering/SimpleNodeRestartIT.java     |  52 ++
 .../DependentControllerServiceIT.java              |  80 +++
 .../tests/system/loadbalance/LoadBalanceIT.java    | 277 +++++++++
 .../parameters/ClusteredParameterContextIT.java    |  23 +-
 .../system/parameters/ParameterContextIT.java      | 357 +++++++++++
 .../system/rpg/ClusteredRemoteProcessGroupIT.java  |  23 +-
 .../tests/system/rpg/RemoteProcessGroupIT.java     | 113 ++++
 .../resources/conf/clustered/node1/bootstrap.conf  |  33 +
 .../resources/conf/clustered/node1/logback.xml     | 177 ++++++
 .../resources/conf/clustered/node1/nifi.properties | 256 ++++++++
 .../conf/clustered/node1/state-management.xml      |  32 +
 .../conf/clustered/node1/zookeeper.properties      |  45 ++
 .../resources/conf/clustered/node2/bootstrap.conf  |  33 +
 .../resources/conf/clustered/node2/logback.xml     | 177 ++++++
 .../resources/conf/clustered/node2/nifi.properties | 256 ++++++++
 .../conf/clustered/node2/state-management.xml      |  32 +
 .../src/test/resources/conf/default/bootstrap.conf |  33 +
 .../src/test/resources/conf/default/logback.xml    | 178 ++++++
 .../test/resources/conf/default/nifi.properties    | 256 ++++++++
 .../resources/conf/default/state-management.xml    |  32 +
 .../resources/conf/default/zookeeper.properties    |  45 ++
 .../src/test/resources/keystore.jks                | Bin 0 -> 3095 bytes
 .../src/test/resources/truststore.jks              | Bin 0 -> 911 bytes
 nifi-system-tests/pom.xml                          |  33 +
 .../toolkit/cli/impl/client/NiFiClientFactory.java |  96 +++
 .../cli/impl/client/nifi/ConnectionClient.java     |  40 ++
 .../impl/client/nifi/ControllerServicesClient.java |  11 +
 ...llerServicesClient.java => CountersClient.java} |  13 +-
 .../toolkit/cli/impl/client/nifi/FlowClient.java   |   8 +
 ...lerServicesClient.java => InputPortClient.java} |  18 +-
 .../toolkit/cli/impl/client/nifi/NiFiClient.java   |  48 ++
 ...erServicesClient.java => OutputPortClient.java} |  19 +-
 .../cli/impl/client/nifi/ProcessGroupClient.java   |   1 -
 .../cli/impl/client/nifi/ProcessorClient.java      |  45 ++
 ...esClient.java => RemoteProcessGroupClient.java} |  18 +-
 .../impl/client/nifi/impl/CRUDJerseyClient.java    | 119 ++++
 .../client/nifi/impl/JerseyConnectionClient.java   | 187 ++++++
 .../nifi/impl/JerseyControllerServicesClient.java  | 106 +++-
 .../client/nifi/impl/JerseyCountersClient.java     |  47 ++
 .../impl/client/nifi/impl/JerseyFlowClient.java    |  12 +
 .../client/nifi/impl/JerseyInputPortClient.java    |  88 +++
 .../impl/client/nifi/impl/JerseyNiFiClient.java    | 111 +++-
 .../client/nifi/impl/JerseyOutputPortClient.java   |  88 +++
 .../client/nifi/impl/JerseyProcessorClient.java    | 172 ++++++
 .../nifi/impl/JerseyRemoteProcessGroupClient.java  | 145 +++++
 pom.xml                                            |  80 +--
 83 files changed, 7026 insertions(+), 397 deletions(-)

diff --git a/.travis.yml b/.travis.yml
index 14a0da6..c6e431c 100644
--- a/.travis.yml
+++ b/.travis.yml
@@ -6,7 +6,7 @@
 # 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.
@@ -47,7 +47,7 @@ matrix:
       jdk: openjdk8
       script: >-
         mvn clean install -V -T 1C
-        -pl `find . -type d \( -name "*-nar" -or -name "*-assembly" -or -name "*hive3*" \) -and -not -name "*api-nar" -and -not -path "*/target/*" -and -not -name "*__*" -printf "!./%P,"`
+        -pl `find . -type d \( -name "*-nar" -or -name "*-assembly" -or -name "*hive3*" -or -name "nifi-system-test*" \) -and -not -name "*api-nar" -and -not -path "*/target/*" -and -not -name "*__*" -printf "!./%P,"`
         -Pcontrib-check,include-grpc -Ddir-only
         -Dmaven.surefire.arguments="-Duser.language=en -Duser.region=US"
         | grep -v -F -f .travis-output-filters
@@ -57,7 +57,7 @@ matrix:
       jdk: openjdk8
       script: >-
         mvn clean install -V -T 1C
-        -pl `find . -type d \( -name "*-nar" -or -name "*-assembly" -or -name "*hive3*" \) -and -not -name "*api-nar" -and -not -path "*/target/*" -and -not -name "*__*" -printf "!./%P,"`
+        -pl `find . -type d \( -name "*-nar" -or -name "*-assembly" -or -name "*hive3*" -or -name "nifi-system-test*" \) -and -not -name "*api-nar" -and -not -path "*/target/*" -and -not -name "*__*" -printf "!./%P,"`
         -Pcontrib-check,include-grpc -Ddir-only
         -Dmaven.surefire.arguments="-Duser.language=fr -Duser.region=FR"
         | grep -v -F -f .travis-output-filters
@@ -67,7 +67,7 @@ matrix:
       jdk: openjdk8
       script: >-
         mvn clean install -V -T 1C
-        -pl `find . -type d \( -name "*-nar" -or -name "*-assembly" -or -name "*hive3*" \) -and -not -name "*api-nar" -and -not -path "*/target/*" -and -not -name "*__*" -printf "!./%P,"`
+        -pl `find . -type d \( -name "*-nar" -or -name "*-assembly" -or -name "*hive3*" -or -name "nifi-system-test*" \) -and -not -name "*api-nar" -and -not -path "*/target/*" -and -not -name "*__*" -printf "!./%P,"`
         -Pcontrib-check,include-grpc -Ddir-only
         -Dmaven.surefire.arguments="-Duser.language=ja -Duser.region=JP"
         | grep -v -F -f .travis-output-filters
@@ -87,7 +87,7 @@ matrix:
       script:
         - >-
           mvn clean install -V -T 1C
-          -pl `find . -type d \( -name "*-nar" -or -name "*-assembly" -or -name "*hive3*" \) -and -not -name "*api-nar" -and -not -path "*/target/*" -and -not -name "*__*" -printf "!./%P,"`
+          -pl `find . -type d \( -name "*-nar" -or -name "*-assembly" -or -name "*hive3*" -or -name "nifi-system-test*" \) -and -not -name "*api-nar" -and -not -path "*/target/*" -and -not -name "*__*" -printf "!./%P,"`
           -Pcontrib-check,include-grpc -Ddir-only
           -Dmaven.surefire.arguments="-Duser.language=en -Duser.region=US"
           | grep -v -F -f .travis-output-filters
diff --git a/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/RunNiFi.java b/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/RunNiFi.java
index 2adefc1..84d1d28 100644
--- a/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/RunNiFi.java
+++ b/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/RunNiFi.java
@@ -234,10 +234,10 @@ public class RunNiFi {
         Integer exitStatus = null;
         switch (cmd.toLowerCase()) {
             case "start":
-                runNiFi.start();
+                runNiFi.start(true);
                 break;
             case "run":
-                runNiFi.start();
+                runNiFi.start(true);
                 break;
             case "stop":
                 runNiFi.stop();
@@ -247,7 +247,7 @@ public class RunNiFi {
                 break;
             case "restart":
                 runNiFi.stop();
-                runNiFi.start();
+                runNiFi.start(true);
                 break;
             case "dump":
                 runNiFi.dump(dumpFile);
@@ -945,7 +945,7 @@ public class RunNiFi {
     }
 
     @SuppressWarnings({"rawtypes", "unchecked"})
-    public void start() throws IOException, InterruptedException {
+    public void start(final boolean monitor) throws IOException {
         final Integer port = getCurrentPort(cmdLogger);
         if (port != null) {
             cmdLogger.info("Apache NiFi is already running, listening to Bootstrap on port " + port);
@@ -1145,8 +1145,6 @@ public class RunNiFi {
         }
 
         shutdownHook = new ShutdownHook(process, this, secretKey, gracefulShutdownSeconds, loggingExecutor);
-        final Runtime runtime = Runtime.getRuntime();
-        runtime.addShutdownHook(shutdownHook);
 
         final String hostname = getHostname();
         final SimpleDateFormat sdf = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss.SSS");
@@ -1157,85 +1155,90 @@ public class RunNiFi {
         }
         serviceManager.notify(NotificationType.NIFI_STARTED, "NiFi Started on Host " + hostname, "Hello,\n\nApache NiFi has been started on host " + hostname + " at " + now + " by user " + user);
 
-        while (true) {
-            final boolean alive = isAlive(process);
+        if (monitor) {
+            final Runtime runtime = Runtime.getRuntime();
+            runtime.addShutdownHook(shutdownHook);
 
-            if (alive) {
-                try {
-                    Thread.sleep(1000L);
-                } catch (final InterruptedException ie) {
-                }
-            } else {
-                try {
-                    runtime.removeShutdownHook(shutdownHook);
-                } catch (final IllegalStateException ise) {
-                    // happens when already shutting down
-                }
+            while (true) {
+                final boolean alive = isAlive(process);
 
-                now = sdf.format(System.currentTimeMillis());
-                if (autoRestartNiFi) {
-                    final File statusFile = getStatusFile(defaultLogger);
-                    if (!statusFile.exists()) {
-                        defaultLogger.info("Status File no longer exists. Will not restart NiFi");
-                        return;
+                if (alive) {
+                    try {
+                        Thread.sleep(1000L);
+                    } catch (final InterruptedException ie) {
                     }
-
-                    final File lockFile = getLockFile(defaultLogger);
-                    if (lockFile.exists()) {
-                        defaultLogger.info("A shutdown was initiated. Will not restart NiFi");
-                        return;
+                } else {
+                    try {
+                        runtime.removeShutdownHook(shutdownHook);
+                    } catch (final IllegalStateException ise) {
+                        // happens when already shutting down
                     }
 
-                    final boolean previouslyStarted = getNifiStarted();
-                    if (!previouslyStarted) {
-                        defaultLogger.info("NiFi never started. Will not restart NiFi");
-                        return;
-                    } else {
-                        setNiFiStarted(false);
-                    }
+                    now = sdf.format(System.currentTimeMillis());
+                    if (autoRestartNiFi) {
+                        final File statusFile = getStatusFile(defaultLogger);
+                        if (!statusFile.exists()) {
+                            defaultLogger.info("Status File no longer exists. Will not restart NiFi");
+                            return;
+                        }
 
-                    if (isSensitiveKeyPresent(props)) {
-                        Path sensitiveKeyFile = createSensitiveKeyFile(confDir);
-                        writeSensitiveKeyFile(props, sensitiveKeyFile);
-                    }
+                        final File lockFile = getLockFile(defaultLogger);
+                        if (lockFile.exists()) {
+                            defaultLogger.info("A shutdown was initiated. Will not restart NiFi");
+                            return;
+                        }
 
-                    defaultLogger.warn("Apache NiFi appears to have died. Restarting...");
-                    secretKey = null;
-                    process = builder.start();
-                    handleLogging(process);
+                        final boolean previouslyStarted = getNifiStarted();
+                        if (!previouslyStarted) {
+                            defaultLogger.info("NiFi never started. Will not restart NiFi");
+                            return;
+                        } else {
+                            setNiFiStarted(false);
+                        }
 
-                    pid = OSUtils.getProcessId(process, defaultLogger);
-                    if (pid == null) {
-                        cmdLogger.warn("Launched Apache NiFi but could not obtain the Process ID");
-                    } else {
-                        nifiPid = pid;
-                        final Properties pidProperties = new Properties();
-                        pidProperties.setProperty(PID_KEY, String.valueOf(nifiPid));
-                        savePidProperties(pidProperties, defaultLogger);
-                        cmdLogger.info("Launched Apache NiFi with Process ID " + pid);
-                    }
+                        if (isSensitiveKeyPresent(props)) {
+                            Path sensitiveKeyFile = createSensitiveKeyFile(confDir);
+                            writeSensitiveKeyFile(props, sensitiveKeyFile);
+                        }
+
+                        defaultLogger.warn("Apache NiFi appears to have died. Restarting...");
+                        secretKey = null;
+                        process = builder.start();
+                        handleLogging(process);
+
+                        pid = OSUtils.getProcessId(process, defaultLogger);
+                        if (pid == null) {
+                            cmdLogger.warn("Launched Apache NiFi but could not obtain the Process ID");
+                        } else {
+                            nifiPid = pid;
+                            final Properties pidProperties = new Properties();
+                            pidProperties.setProperty(PID_KEY, String.valueOf(nifiPid));
+                            savePidProperties(pidProperties, defaultLogger);
+                            cmdLogger.info("Launched Apache NiFi with Process ID " + pid);
+                        }
 
-                    shutdownHook = new ShutdownHook(process, this, secretKey, gracefulShutdownSeconds, loggingExecutor);
-                    runtime.addShutdownHook(shutdownHook);
+                        shutdownHook = new ShutdownHook(process, this, secretKey, gracefulShutdownSeconds, loggingExecutor);
+                        runtime.addShutdownHook(shutdownHook);
 
-                    final boolean started = waitForStart();
+                        final boolean started = waitForStart();
 
-                    if (started) {
-                        defaultLogger.info("Successfully started Apache NiFi{}", (pid == null ? "" : " with PID " + pid));
-                        // We are expected to restart nifi, so send a notification that it died. If we are not restarting nifi,
-                        // then this means that we are intentionally stopping the service.
-                        serviceManager.notify(NotificationType.NIFI_DIED, "NiFi Died on Host " + hostname,
+                        if (started) {
+                            defaultLogger.info("Successfully started Apache NiFi{}", (pid == null ? "" : " with PID " + pid));
+                            // We are expected to restart nifi, so send a notification that it died. If we are not restarting nifi,
+                            // then this means that we are intentionally stopping the service.
+                            serviceManager.notify(NotificationType.NIFI_DIED, "NiFi Died on Host " + hostname,
                                 "Hello,\n\nIt appears that Apache NiFi has died on host " + hostname + " at " + now + "; automatically restarting NiFi");
-                    } else {
-                        defaultLogger.error("Apache NiFi does not appear to have started");
-                        // We are expected to restart nifi, so send a notification that it died. If we are not restarting nifi,
-                        // then this means that we are intentionally stopping the service.
-                        serviceManager.notify(NotificationType.NIFI_DIED, "NiFi Died on Host " + hostname,
+                        } else {
+                            defaultLogger.error("Apache NiFi does not appear to have started");
+                            // We are expected to restart nifi, so send a notification that it died. If we are not restarting nifi,
+                            // then this means that we are intentionally stopping the service.
+                            serviceManager.notify(NotificationType.NIFI_DIED, "NiFi Died on Host " + hostname,
                                 "Hello,\n\nIt appears that Apache NiFi has died on host " + hostname + " at " + now +
-                                        ". Attempted to restart NiFi but the services does not appear to have restarted!");
+                                    ". Attempted to restart NiFi but the services does not appear to have restarted!");
+                        }
+                    } else {
+                        return;
                     }
-                } else {
-                    return;
                 }
             }
         }
diff --git a/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/file/FileUtils.java b/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/file/FileUtils.java
index 13b6a4b..0bfc083 100644
--- a/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/file/FileUtils.java
+++ b/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/file/FileUtils.java
@@ -16,6 +16,8 @@
  */
 package org.apache.nifi.util.file;
 
+import org.slf4j.Logger;
+
 import java.io.Closeable;
 import java.io.File;
 import java.io.FileInputStream;
@@ -36,8 +38,6 @@ import java.util.Collection;
 import java.util.List;
 import java.util.Random;
 
-import org.slf4j.Logger;
-
 /**
  * A utility class containing a few useful static methods to do typical IO operations.
  *
@@ -249,7 +249,7 @@ public class FileUtils {
         }
         //now delete the file itself regardless of whether it is plain file or a directory
         if (!FileUtils.deleteFile(file, null, 5)) {
-            throw new IOException("Unable to delete " + file.getAbsolutePath());
+            Files.delete(file.toPath());
         }
     }
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/server/ZooKeeperStateServer.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/server/ZooKeeperStateServer.java
index 81e7194..c553e5a 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/server/ZooKeeperStateServer.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/server/ZooKeeperStateServer.java
@@ -17,13 +17,6 @@
 
 package org.apache.nifi.controller.state.server;
 
-import java.io.BufferedInputStream;
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.Properties;
-
 import org.apache.nifi.util.NiFiProperties;
 import org.apache.zookeeper.server.DatadirCleanupManager;
 import org.apache.zookeeper.server.ServerCnxnFactory;
@@ -38,6 +31,13 @@ import org.apache.zookeeper.server.quorum.QuorumPeerConfig.ConfigException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.BufferedInputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Properties;
+
 public class ZooKeeperStateServer extends ZooKeeperServerMain {
     private static final Logger logger = LoggerFactory.getLogger(ZooKeeperStateServer.class);
 
@@ -82,7 +82,21 @@ public class ZooKeeperStateServer extends ZooKeeperServerMain {
         final ServerConfig config = new ServerConfig();
         config.readFrom(quorumPeerConfig);
         try {
-            transactionLog = new FileTxnSnapLog(config.getDataLogDir(), config.getDataDir());
+            for (int i=0; i < 10; i++) {
+                try {
+                    transactionLog = new FileTxnSnapLog(config.getDataLogDir(), config.getDataDir());
+                    break;
+                } catch (final FileTxnSnapLog.DatadirException dde) {
+                    // The constructor for FileTxnSnapLog sometimes throws a DatadirException indicating that it is unable to create data directory,
+                    // but the data directory already exists. It appears to be a race condition with another ZooKeeper thread. Even if we create the
+                    // directory before entering the constructor, we sometimes see the issue occur. So we just give it up to 10 tries
+                    try {
+                        Thread.sleep(50L);
+                    } catch (final InterruptedException ie) {
+                        Thread.currentThread().interrupt();;
+                    }
+                }
+            }
 
             embeddedZkServer = new ZooKeeperServer();
             embeddedZkServer.setTxnLogFactory(transactionLog);
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java
index f04286a..481c8f4 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java
@@ -549,6 +549,8 @@ public final class StandardProcessGroup implements ProcessGroup {
             inputPorts.put(requireNonNull(port).getIdentifier(), port);
             flowManager.onInputPortAdded(port);
             onComponentModified();
+
+            LOG.info("Input Port {} added to {}", port, this);
         } finally {
             writeLock.unlock();
         }
@@ -624,12 +626,14 @@ public final class StandardProcessGroup implements ProcessGroup {
         writeLock.lock();
         try {
             // Unique port check within the same group.
-            verifyPortUniqueness(port, outputPorts, name -> getOutputPortByName(name));
+            verifyPortUniqueness(port, outputPorts, this::getOutputPortByName);
 
             port.setProcessGroup(this);
             outputPorts.put(port.getIdentifier(), port);
             flowManager.onOutputPortAdded(port);
             onComponentModified();
+
+            LOG.info("Output Port {} added to {}", port, this);
         } finally {
             writeLock.unlock();
         }
@@ -703,6 +707,8 @@ public final class StandardProcessGroup implements ProcessGroup {
             group.findAllProcessors().forEach(this::updateControllerServiceReferences);
 
             onComponentModified();
+
+            LOG.info("{} added to {}", group, this);
         } finally {
             writeLock.unlock();
         }
@@ -801,6 +807,8 @@ public final class StandardProcessGroup implements ProcessGroup {
             remoteGroup.setProcessGroup(this);
             remoteGroups.put(Objects.requireNonNull(remoteGroup).getIdentifier(), remoteGroup);
             onComponentModified();
+
+            LOG.info("{} added to {}", remoteGroup, this);
         } finally {
             writeLock.unlock();
         }
@@ -884,6 +892,8 @@ public final class StandardProcessGroup implements ProcessGroup {
             flowManager.onProcessorAdded(processor);
             updateControllerServiceReferences(processor);
             onComponentModified();
+
+            LOG.info("{} added to {}", processor, this);
         } finally {
             writeLock.unlock();
         }
@@ -1960,6 +1970,7 @@ public final class StandardProcessGroup implements ProcessGroup {
             }
 
             onComponentModified();
+            LOG.info("{} added to {}", funnel, this);
         } finally {
             writeLock.unlock();
         }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/remote/StandardRemoteProcessGroup.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/remote/StandardRemoteProcessGroup.java
index 8514b5d..55e5109 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/remote/StandardRemoteProcessGroup.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/remote/StandardRemoteProcessGroup.java
@@ -16,37 +16,6 @@
  */
 package org.apache.nifi.remote;
 
-import static java.util.Objects.requireNonNull;
-
-import java.io.IOException;
-import java.net.InetAddress;
-import java.net.NetworkInterface;
-import java.nio.charset.StandardCharsets;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Date;
-import java.util.Enumeration;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.LinkedHashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Optional;
-import java.util.Set;
-import java.util.UUID;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReadWriteLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-import java.util.function.Function;
-import java.util.stream.Collectors;
-import javax.net.ssl.SSLContext;
-import javax.ws.rs.core.Response;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.authorization.Resource;
 import org.apache.nifi.authorization.resource.Authorizable;
@@ -81,6 +50,38 @@ import org.apache.nifi.web.api.dto.PortDTO;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import javax.net.ssl.SSLContext;
+import javax.ws.rs.core.Response;
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.NetworkInterface;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.Enumeration;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+import static java.util.Objects.requireNonNull;
+
 /**
  * Represents the Root Process Group of a remote NiFi Instance. Holds
  * information about that remote instance, as well as Incoming Ports and
@@ -463,6 +464,8 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
     public void setInputPorts(final Set<RemoteProcessGroupPortDescriptor> ports, final boolean pruneUnusedPorts) {
         writeLock.lock();
         try {
+            logger.debug("Updating Input Ports for {}", this);
+
             final List<String> newPortTargetIds = new ArrayList<>();
             for (final RemoteProcessGroupPortDescriptor descriptor : ports) {
                 newPortTargetIds.add(descriptor.getTargetId());
@@ -483,8 +486,11 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
                     sendPort = inputPortByName.get(descriptor.getName());
                     if (sendPort == null) {
                         sendPort = addInputPort(descriptor);
+                        logger.info("Added Input Port {} with Name {} and Target Identifier {} to {}", sendPort.getIdentifier(), sendPort.getName(), sendPort.getTargetIdentifier(), this);
                     } else {
+                        final String previousTargetId = sendPort.getTargetIdentifier();
                         sendPort.setTargetIdentifier(descriptor.getTargetId());
+                        logger.info("Updated Target identifier for Input Port with Name {} from {} to {} for {}", descriptor.getName(), previousTargetId, descriptor.getTargetId(), this);
                     }
                 }
 
@@ -501,6 +507,8 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
             // a ConcurrentModificationException.
             if (pruneUnusedPorts) {
                 final Iterator<StandardRemoteGroupPort> itr = inputPorts.values().iterator();
+
+                int prunedCount = 0;
                 while (itr.hasNext()) {
                     final StandardRemoteGroupPort port = itr.next();
                     if (!newPortTargetIds.contains(port.getTargetIdentifier())) {
@@ -510,9 +518,19 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
                         // If port has incoming connection, it will be cleaned up when the connection is removed
                         if (!port.hasIncomingConnection()) {
                             itr.remove();
+                            logger.debug("Pruning unused Input Port {} from {}", port, this);
+                            prunedCount++;
                         }
                     }
                 }
+
+                if (prunedCount == 0) {
+                    logger.debug("There were no Input Ports to prune from {}", this);
+                } else {
+                    logger.debug("Successfully pruned {} unused Input Ports from {}", prunedCount, this);
+                }
+            } else {
+                logger.debug("Updated Input Ports for {} but did not attempt to prune any unused ports", this);
             }
         } finally {
             writeLock.unlock();
@@ -572,8 +590,11 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
                     receivePort = outputPortByName.get(descriptor.getName());
                     if (receivePort == null) {
                         receivePort = addOutputPort(descriptor);
+                        logger.info("Added Output Port {} with Name {} and Target Identifier {} to {}", receivePort.getIdentifier(), receivePort.getName(), receivePort.getTargetIdentifier(), this);
                     } else {
+                        final String previousTargetId = receivePort.getTargetIdentifier();
                         receivePort.setTargetIdentifier(descriptor.getTargetId());
+                        logger.info("Updated Target identifier for Output Port with Name {} from {} to {} for {}", descriptor.getName(), previousTargetId, descriptor.getTargetId(), this);
                     }
                 }
 
@@ -590,6 +611,8 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
             // a ConcurrentModificationException.
             if (pruneUnusedPorts) {
                 final Iterator<StandardRemoteGroupPort> itr = outputPorts.values().iterator();
+
+                int prunedCount = 0;
                 while (itr.hasNext()) {
                     final StandardRemoteGroupPort port = itr.next();
                     if (!newPortTargetIds.contains(port.getTargetIdentifier())) {
@@ -599,10 +622,20 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
                         // If port has connections, it will be cleaned up when connections are removed
                         if (port.getConnections().isEmpty()) {
                             itr.remove();
+                            logger.info("Pruning unused Output Port {} from {}", port, this);
                         }
                     }
                 }
+
+                if (prunedCount == 0) {
+                    logger.debug("There were no Output Ports to prune from {}", this);
+                } else {
+                    logger.debug("Successfully pruned {} unused Output Ports from {}", prunedCount, this);
+                }
+            } else {
+                logger.debug("Updated Output Ports for {} but did not attempt to prune any unused ports", this);
             }
+
         } finally {
             writeLock.unlock();
         }
@@ -882,6 +915,16 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
                 final RemoteProcessGroupCounts newCounts = new RemoteProcessGroupCounts(inputPortCount, outputPortCount);
                 setCounts(newCounts);
                 this.refreshContentsTimestamp = System.currentTimeMillis();
+
+                final List<String> inputPortString = dto.getInputPorts().stream()
+                    .map(port -> "InputPort[name=" + port.getName() + ", targetId=" + port.getId() + "]")
+                    .collect(Collectors.toList());
+                final List<String> outputPortString = dto.getInputPorts().stream()
+                    .map(port -> "OutputPort[name=" + port.getName() + ", targetId=" + port.getId() + "]")
+                    .collect(Collectors.toList());
+
+                logger.info("Successfully refreshed Flow Contents for {}; updated to reflect {} Input Ports {} and {} Output Ports {}", this, dto.getInputPorts().size(), inputPortString,
+                    dto.getOutputPorts().size(), outputPortString);
             } finally {
                 writeLock.unlock();
             }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/groovy/org/apache/nifi/controller/StandardFlowServiceSpec.groovy b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/groovy/org/apache/nifi/controller/StandardFlowServiceSpec.groovy
deleted file mode 100644
index 7ffe21d..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/groovy/org/apache/nifi/controller/StandardFlowServiceSpec.groovy
+++ /dev/null
@@ -1,130 +0,0 @@
-/*
- * 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.
- */
-package org.apache.nifi.controller
-
-import org.apache.nifi.authorization.Authorizer
-import org.apache.nifi.cluster.coordination.ClusterCoordinator
-import org.apache.nifi.cluster.coordination.node.NodeConnectionState
-import org.apache.nifi.cluster.coordination.node.NodeConnectionStatus
-import org.apache.nifi.cluster.coordination.node.OffloadCode
-import org.apache.nifi.cluster.protocol.NodeIdentifier
-import org.apache.nifi.cluster.protocol.impl.NodeProtocolSenderListener
-import org.apache.nifi.cluster.protocol.message.OffloadMessage
-import org.apache.nifi.components.state.Scope
-import org.apache.nifi.components.state.StateManager
-import org.apache.nifi.components.state.StateManagerProvider
-import org.apache.nifi.connectable.Connection
-import org.apache.nifi.controller.queue.FlowFileQueue
-import org.apache.nifi.controller.status.ProcessGroupStatus
-import org.apache.nifi.encrypt.StringEncryptor
-import org.apache.nifi.groups.ProcessGroup
-import org.apache.nifi.groups.RemoteProcessGroup
-import org.apache.nifi.state.MockStateMap
-import org.apache.nifi.util.NiFiProperties
-import org.apache.nifi.web.revision.RevisionManager
-import org.junit.Ignore
-import spock.lang.Specification
-import spock.util.concurrent.BlockingVariable
-
-import java.util.concurrent.TimeUnit
-
-@Ignore("Problematic unit test that expects internals of StandardFlowService not to change, as it dictates the order in which methods are called internally")
-class StandardFlowServiceSpec extends Specification {
-    def "handle an OffloadMessage"() {
-        given: 'a node to offload'
-        def nodeToOffload = createNodeIdentifier 1
-
-        and: 'a simple flow with one root group and a single processor'
-        def stateManager = Mock StateManager
-        def stateMap = new MockStateMap([:], 1)
-        stateManager.getState(_ as Scope) >> stateMap
-        def stateManagerProvider = Mock StateManagerProvider
-        stateManagerProvider.getStateManager(_ as String) >> stateManager
-
-        def rootGroup = Mock ProcessGroup
-        def flowController = Mock FlowController
-        flowController.getStateManagerProvider() >> stateManagerProvider
-        _ * flowController.rootGroup >> rootGroup
-
-        def clusterCoordinator = Mock ClusterCoordinator
-
-        def processGroupStatus = Mock ProcessGroupStatus
-        def processorNode = Mock ProcessorNode
-        def remoteProcessGroup = Mock RemoteProcessGroup
-        def flowFileQueue = Mock FlowFileQueue
-        def connection = Mock Connection
-
-        and: 'a flow service to handle the OffloadMessage'
-        def flowService = StandardFlowService.createClusteredInstance(flowController, NiFiProperties.createBasicNiFiProperties('src/test/resources/conf/nifi.properties',
-                [(NiFiProperties.CLUSTER_NODE_PROTOCOL_PORT): nodeToOffload.socketPort as String,
-                 (NiFiProperties.WEB_HTTP_PORT)             : nodeToOffload.apiPort as String,
-                 (NiFiProperties.LOAD_BALANCE_PORT)         : nodeToOffload.loadBalancePort as String]),
-                Mock(NodeProtocolSenderListener), clusterCoordinator, Mock(StringEncryptor), Mock(RevisionManager), Mock(Authorizer))
-
-        def waitForFinishOffload = new BlockingVariable(5, TimeUnit.SECONDS)//new CountDownLatch(1)
-
-        when: 'the flow services receives an OffloadMessage'
-        flowService.handle(new OffloadMessage(nodeId: nodeToOffload, explanation: 'unit test offload'), [] as Set)
-        waitForFinishOffload.get()
-
-        then: 'no exceptions are thrown'
-        noExceptionThrown()
-
-        and: 'the connection status for the node in the flow controller is set to OFFLOADING'
-        1 * flowController.setConnectionStatus({ NodeConnectionStatus status ->
-            status.nodeIdentifier.logicallyEquals(nodeToOffload) && status.state == NodeConnectionState.OFFLOADING && status.offloadCode == OffloadCode.OFFLOADED
-        } as NodeConnectionStatus)
-
-//        then: 'all processors are requested to stop'
-//        1 * flowController.stopAllProcessors()
-
-        then: 'all processors are requested to terminate'
-        1 * processorNode.scheduledState >> ScheduledState.STOPPED
-        1 * processorNode.processGroup >> rootGroup
-        1 * rootGroup.terminateProcessor({ ProcessorNode pn -> pn == processorNode } as ProcessorNode)
-        1 * rootGroup.findAllProcessors() >> [processorNode]
-
-        then: 'all remote process groups are requested to terminate'
-        1 * remoteProcessGroup.stopTransmitting()
-        1 * rootGroup.findAllRemoteProcessGroups() >> [remoteProcessGroup]
-
-        then: 'all queues are requested to offload'
-        1 * flowFileQueue.offloadQueue()
-
-        then: 'the queued count in the flow controller status is 0 to allow the offloading code to to complete'
-        1 * flowController.getControllerStatus() >> processGroupStatus
-        1 * processGroupStatus.getQueuedCount() >> 0
-
-        then: 'all queues are requested to reset to the original partitioner for the load balancing strategy'
-        1 * flowFileQueue.resetOffloadedQueue()
-
-        then: 'the connection status for the node in the flow controller is set to OFFLOADED'
-        1 * flowController.setConnectionStatus({ NodeConnectionStatus status ->
-            status.nodeIdentifier.logicallyEquals(nodeToOffload) && status.state == NodeConnectionState.OFFLOADED && status.offloadCode == OffloadCode.OFFLOADED
-        } as NodeConnectionStatus)
-
-        then: 'the cluster coordinator is requested to finish the node offload'
-        1 * clusterCoordinator.finishNodeOffload({ NodeIdentifier nodeIdentifier ->
-            nodeIdentifier.logicallyEquals(nodeToOffload)
-        } as NodeIdentifier) >> { waitForFinishOffload.set(it) }
-    }
-
-    private static NodeIdentifier createNodeIdentifier(final int index) {
-        new NodeIdentifier("node-id-$index", "localhost", 8000 + index, "localhost", 9000 + index,
-                "localhost", 10000 + index, 11000 + index, false)
-    }
-}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/versioned/ImportFlowIT.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/versioned/ImportFlowIT.java
index 996d363..c4fec3e 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/versioned/ImportFlowIT.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/versioned/ImportFlowIT.java
@@ -162,7 +162,7 @@ public class ImportFlowIT extends FrameworkIntegrationTest {
         processor.setProperties(Collections.singletonMap(UsernamePasswordProcessor.PASSWORD.getName(), "#{secret-parameter}"));
         differences = getLocalModifications(innerGroup, versionedFlowWithExplicitValue);
         assertEquals(1, differences.size());
-        assertEquals(DifferenceType.PROPERTY_ADDED, differences.iterator().next().getDifferenceType());
+        assertEquals(DifferenceType.PROPERTY_PARAMETERIZED, differences.iterator().next().getDifferenceType());
 
         // Create a Versioned Flow that contains the Parameter Reference.
         final VersionedFlowSnapshot versionedFlowWithParameterReference = createFlowSnapshot(Collections.emptyList(), Collections.singletonList(processor), null);
@@ -174,7 +174,7 @@ public class ImportFlowIT extends FrameworkIntegrationTest {
         processor.setProperties(Collections.singletonMap(UsernamePasswordProcessor.PASSWORD.getName(), "secret"));
         differences = getLocalModifications(innerGroup, versionedFlowWithParameterReference);
         assertEquals(1, differences.size());
-        assertEquals(DifferenceType.PROPERTY_REMOVED, differences.iterator().next().getDifferenceType());
+        assertEquals(DifferenceType.PROPERTY_PARAMETERIZATION_REMOVED, differences.iterator().next().getDifferenceType());
     }
 
     @Test
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/SystemBundle.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/SystemBundle.java
index 0fb2bad..9f6b88c 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/SystemBundle.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/SystemBundle.java
@@ -32,6 +32,10 @@ public final class SystemBundle {
     public static final BundleCoordinate SYSTEM_BUNDLE_COORDINATE = new BundleCoordinate(
             BundleCoordinate.DEFAULT_GROUP, "system", BundleCoordinate.DEFAULT_VERSION);
 
+    public static Bundle create(final NiFiProperties niFiProperties) {
+        return create(niFiProperties, ClassLoader.getSystemClassLoader());
+    }
+
     /**
      * Returns a bundle representing the system class loader.
      *
@@ -39,9 +43,7 @@ public final class SystemBundle {
      *                       which will become the working directory of the returned bundle
      * @return a bundle for the system class loader
      */
-    public static Bundle create(final NiFiProperties niFiProperties) {
-        final ClassLoader systemClassLoader = ClassLoader.getSystemClassLoader();
-
+    public static Bundle create(final NiFiProperties niFiProperties, final ClassLoader systemClassLoader) {
         final String narLibraryDirectory = niFiProperties.getProperty(NiFiProperties.NAR_LIBRARY_DIRECTORY);
         if (StringUtils.isBlank(narLibraryDirectory)) {
             throw new IllegalStateException("Unable to create system bundle because " + NiFiProperties.NAR_LIBRARY_DIRECTORY + " was null or empty");
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/main/java/org/apache/nifi/BootstrapListener.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/main/java/org/apache/nifi/BootstrapListener.java
index dc5f1dd..c0dbcbc 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/main/java/org/apache/nifi/BootstrapListener.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/main/java/org/apache/nifi/BootstrapListener.java
@@ -179,6 +179,7 @@ public class BootstrapListener {
                                     case SHUTDOWN:
                                         logger.info("Received SHUTDOWN request from Bootstrap");
                                         echoShutdown(socket.getOutputStream());
+                                        socket.close();
                                         nifi.shutdownHook();
                                         return;
                                     case DUMP:
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/main/java/org/apache/nifi/EmbeddedNiFi.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/main/java/org/apache/nifi/EmbeddedNiFi.java
index 790947c..6f1634e 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/main/java/org/apache/nifi/EmbeddedNiFi.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/main/java/org/apache/nifi/EmbeddedNiFi.java
@@ -35,6 +35,13 @@ public class EmbeddedNiFi extends NiFi {
         super(convertArgumentsToValidatedNiFiProperties(args), rootClassLoader);
     }
 
+    public EmbeddedNiFi(String[] args, ClassLoader rootClassLoader, ClassLoader bootstrapClassLoader)
+        throws ClassNotFoundException, IOException, NoSuchMethodException,
+        InstantiationException, IllegalAccessException, IllegalArgumentException, InvocationTargetException {
+
+        super(convertArgumentsToValidatedNiFiProperties(args, bootstrapClassLoader), rootClassLoader);
+    }
+
     @Override
     protected void initLogging() {
         // do nothing when running in embedded mode
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/main/java/org/apache/nifi/NiFi.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/main/java/org/apache/nifi/NiFi.java
index 29578e0..acfed37 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/main/java/org/apache/nifi/NiFi.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/main/java/org/apache/nifi/NiFi.java
@@ -123,7 +123,7 @@ public class NiFi {
         // redirect JUL log events
         initLogging();
 
-        final Bundle systemBundle = SystemBundle.create(properties);
+        final Bundle systemBundle = SystemBundle.create(properties, rootClassLoader);
 
         // expand the nars
         final ExtensionMapping extensionMapping = NarUnpacker.unpackNars(properties, systemBundle);
@@ -305,8 +305,11 @@ public class NiFi {
     }
 
     protected static NiFiProperties convertArgumentsToValidatedNiFiProperties(String[] args) {
-        final ClassLoader bootstrap = createBootstrapClassLoader();
-        NiFiProperties properties = initializeProperties(args, bootstrap);
+        return convertArgumentsToValidatedNiFiProperties(args, createBootstrapClassLoader());
+    }
+
+    protected static NiFiProperties convertArgumentsToValidatedNiFiProperties(String[] args, final ClassLoader bootstrapClassLoader) {
+        NiFiProperties properties = initializeProperties(args, bootstrapClassLoader);
         properties.validate();
         return properties;
     }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.java
index bd2687e..95f5c2e 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.java
@@ -71,10 +71,6 @@ import java.util.concurrent.atomic.AtomicReference;
 public class StandardRemoteGroupPort extends RemoteGroupPort {
 
     private static final long BATCH_SEND_NANOS = TimeUnit.MILLISECONDS.toNanos(500L); // send batches of up to 500 millis
-    public static final String USER_AGENT = "NiFi-Site-to-Site";
-    public static final String CONTENT_TYPE = "application/octet-stream";
-
-    public static final int GZIP_COMPRESSION_LEVEL = 1;
 
     private static final String CATEGORY = "Site to Site";
 
@@ -88,7 +84,6 @@ public class StandardRemoteGroupPort extends RemoteGroupPort {
     private final AtomicBoolean targetRunning = new AtomicBoolean(true);
     private final SSLContext sslContext;
     private final TransferDirection transferDirection;
-    private final NiFiProperties nifiProperties;
     private volatile String targetId;
 
     private final AtomicReference<SiteToSiteClient> clientRef = new AtomicReference<>();
@@ -109,7 +104,6 @@ public class StandardRemoteGroupPort extends RemoteGroupPort {
         this.remoteGroup = remoteGroup;
         this.transferDirection = direction;
         this.sslContext = sslContext;
-        this.nifiProperties = nifiProperties;
         setScheduldingPeriod(MINIMUM_SCHEDULING_NANOS + " nanos");
     }
 
@@ -157,7 +151,7 @@ public class StandardRemoteGroupPort extends RemoteGroupPort {
             try {
                 client.close();
             } catch (final IOException ioe) {
-                logger.warn("Failed to properly shutdown Site-to-Site Client due to {}", ioe);
+                logger.warn("Failed to properly shutdown Site-to-Site Client", ioe);
             }
         }
     }
@@ -226,7 +220,7 @@ public class StandardRemoteGroupPort extends RemoteGroupPort {
         }
 
         final SiteToSiteClient client = getSiteToSiteClient();
-        Transaction transaction = null;
+        final Transaction transaction;
         try {
             transaction = client.createTransaction(transferDirection);
         } catch (final NoValidPeerException e) {
diff --git a/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions-nar/pom.xml b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions-nar/pom.xml
new file mode 100644
index 0000000..a2f76aa
--- /dev/null
+++ b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions-nar/pom.xml
@@ -0,0 +1,43 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>nifi-system-test-extensions-bundle</artifactId>
+        <version>1.11.0-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>nifi-system-test-extensions-nar</artifactId>
+    <version>1.11.0-SNAPSHOT</version>
+    <packaging>nar</packaging>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-system-test-extensions</artifactId>
+            <version>1.11.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-standard-services-api-nar</artifactId>
+            <version>1.11.0-SNAPSHOT</version>
+            <type>nar</type>
+        </dependency>
+    </dependencies>
+
+</project>
diff --git a/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/pom.xml b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/pom.xml
new file mode 100644
index 0000000..97170bb
--- /dev/null
+++ b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/pom.xml
@@ -0,0 +1,41 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <artifactId>nifi-system-test-extensions-bundle</artifactId>
+        <groupId>org.apache.nifi</groupId>
+        <version>1.11.0-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+
+    <artifactId>nifi-system-test-extensions</artifactId>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+            <version>1.11.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-utils</artifactId>
+            <version>1.11.0-SNAPSHOT</version>
+        </dependency>
+    </dependencies>
+
+</project>
\ No newline at end of file
diff --git a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/ControllerServicesClient.java b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/cs/tests/system/SleepService.java
similarity index 57%
copy from nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/ControllerServicesClient.java
copy to nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/cs/tests/system/SleepService.java
index af55421..f6923b4 100644
--- a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/ControllerServicesClient.java
+++ b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/cs/tests/system/SleepService.java
@@ -14,20 +14,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.nifi.toolkit.cli.impl.client.nifi;
+package org.apache.nifi.cs.tests.system;
 
-import org.apache.nifi.web.api.entity.ControllerServiceEntity;
-import org.apache.nifi.web.api.entity.ControllerServiceRunStatusEntity;
-
-import java.io.IOException;
-
-/**
- * Client for interacting with NiFi's Controller Services Resource.
- */
-public interface ControllerServicesClient {
-
-    ControllerServiceEntity getControllerService(String id) throws NiFiClientException, IOException;
-
-    ControllerServiceEntity activateControllerService(String id, ControllerServiceRunStatusEntity runStatusEntity) throws NiFiClientException, IOException;
+import org.apache.nifi.controller.ControllerService;
 
+public interface SleepService extends ControllerService {
+    void sleep();
 }
diff --git a/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/cs/tests/system/StandardSleepService.java b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/cs/tests/system/StandardSleepService.java
new file mode 100644
index 0000000..27ffca4
--- /dev/null
+++ b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/cs/tests/system/StandardSleepService.java
@@ -0,0 +1,118 @@
+/*
+ * 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.
+ */
+package org.apache.nifi.cs.tests.system;
+
+import org.apache.nifi.annotation.lifecycle.OnDisabled;
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.processor.util.StandardValidators;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+public class StandardSleepService extends AbstractControllerService implements SleepService {
+    public static final PropertyDescriptor VALIDATE_SLEEP_TIME = new PropertyDescriptor.Builder()
+        .name("Validate Sleep Time")
+        .description("The amount of time to sleep during validation")
+        .required(false)
+        .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+        .defaultValue("0 sec")
+        .build();
+    public static final PropertyDescriptor TRIGGER_SLEEP_TIME = new PropertyDescriptor.Builder()
+        .name("Trigger Sleep Time")
+        .description("The amount of time to sleep during each trigger")
+        .required(false)
+        .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+        .defaultValue("0 sec")
+        .build();
+    public static final PropertyDescriptor ON_ENABLED_SLEEP_TIME = new PropertyDescriptor.Builder()
+        .name("@OnEnabled Sleep Time")
+        .description("The amount of time to sleep when enabled")
+        .required(false)
+        .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+        .defaultValue("0 sec")
+        .build();
+    public static final PropertyDescriptor ON_DISABLED_SLEEP_TIME = new PropertyDescriptor.Builder()
+        .name("@OnDisabled Sleep Time")
+        .description("The amount of time to sleep when disabeld")
+        .required(false)
+        .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+        .defaultValue("0 sec")
+        .build();
+    public static final PropertyDescriptor DEPENDENT_SERVICE = new PropertyDescriptor.Builder()
+        .name("Dependent Service")
+        .description("Another Controller Service that this one depends on. This is helpful for testing when Service A depends on Service B how enabling/disabling/etc. work")
+        .required(false)
+        .identifiesControllerService(SleepService.class)
+        .build();
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        final List<PropertyDescriptor> properties = new ArrayList<>();
+        properties.add(VALIDATE_SLEEP_TIME);
+        properties.add(ON_ENABLED_SLEEP_TIME);
+        properties.add(TRIGGER_SLEEP_TIME);
+        properties.add(ON_DISABLED_SLEEP_TIME);
+        properties.add(DEPENDENT_SERVICE);
+        return properties;
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
+        final long sleepMillis = validationContext.getProperty(VALIDATE_SLEEP_TIME).asTimePeriod(TimeUnit.MILLISECONDS);
+        sleep(sleepMillis);
+
+        return Collections.emptyList();
+    }
+
+    private void sleep(final long millis) {
+        if (millis > 0L) {
+            try {
+                Thread.sleep(millis);
+            } catch (final InterruptedException ie) {
+                Thread.currentThread().interrupt();
+            }
+        }
+    }
+
+    @OnEnabled
+    public void onEnabled(final ConfigurationContext context) {
+        sleep(context.getProperty(ON_ENABLED_SLEEP_TIME).asTimePeriod(TimeUnit.MILLISECONDS));
+    }
+
+    @OnDisabled
+    public void onDisabled(final ConfigurationContext context) {
+        sleep(context.getProperty(ON_DISABLED_SLEEP_TIME).asTimePeriod(TimeUnit.MILLISECONDS));
+    }
+
+    @Override
+    public void sleep() {
+        sleep(getConfigurationContext().getProperty(TRIGGER_SLEEP_TIME).asTimePeriod(TimeUnit.MILLISECONDS));
+
+        final SleepService dependentService = getConfigurationContext().getProperty(DEPENDENT_SERVICE).asControllerService(SleepService.class);
+        if (dependentService != null) {
+            dependentService.sleep();
+        }
+    }
+}
diff --git a/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/processors/tests/system/CountEvents.java b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/processors/tests/system/CountEvents.java
new file mode 100644
index 0000000..64aaf84
--- /dev/null
+++ b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/processors/tests/system/CountEvents.java
@@ -0,0 +1,97 @@
+/*
+ * 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.
+ */
+package org.apache.nifi.processors.tests.system;
+
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.PropertyDescriptor.Builder;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.AbstractSessionFactoryProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+import org.apache.nifi.processor.exception.ProcessException;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.apache.nifi.components.Validator.VALID;
+
+@SupportsBatching
+public class CountEvents extends AbstractSessionFactoryProcessor {
+    private volatile ProcessSessionFactory sessionFactory;
+    private final AtomicBoolean firstScheduleCounted = new AtomicBoolean(false);
+
+    static final PropertyDescriptor NAME = new Builder()
+        .name("Name")
+        .displayName("Name")
+        .description("Arbitrary Name")
+        .required(false)
+        .addValidator(VALID)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .build();
+
+    static final PropertyDescriptor SENSITIVE = new Builder()
+        .name("Sensitive")
+        .displayName("Sensitive")
+        .description("Sensitive Property with no real meaning")
+        .required(false)
+        .addValidator(VALID)
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+        .sensitive(true)
+        .build();
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return Arrays.asList(NAME, SENSITIVE);
+    }
+
+    @OnStopped
+    public void onStopped() {
+        sessionFactory.createSession().adjustCounter("Stopped", 1, true);
+    }
+
+    @OnScheduled
+    public void onScheduled() {
+        if (sessionFactory != null) {
+            sessionFactory.createSession().adjustCounter("Scheduled", 1, true);
+        }
+    }
+
+    @OnUnscheduled
+    public void onUnScheduled() {
+        if (sessionFactory != null) {
+            sessionFactory.createSession().adjustCounter("UnScheduled", 1, true);
+        }
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSessionFactory sessionFactory) throws ProcessException {
+        this.sessionFactory = sessionFactory;
+
+        final ProcessSession session = sessionFactory.createSession();
+        if (!firstScheduleCounted.getAndSet(true)) {
+            session.adjustCounter("Scheduled", 1, true);
+        }
+
+        session.adjustCounter("Triggered", 1, true);
+    }
+}
diff --git a/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/processors/tests/system/GenerateFlowFile.java b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/processors/tests/system/GenerateFlowFile.java
new file mode 100644
index 0000000..70c71a4
--- /dev/null
+++ b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/processors/tests/system/GenerateFlowFile.java
@@ -0,0 +1,136 @@
+/*
+ * 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.
+ */
+package org.apache.nifi.processors.tests.system;
+
+import org.apache.nifi.annotation.configuration.DefaultSchedule;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.AttributeExpression;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.DataUnit;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.Set;
+
+
+@DefaultSchedule(period = "10 mins")
+public class GenerateFlowFile extends AbstractProcessor {
+    public static final PropertyDescriptor FILE_SIZE = new PropertyDescriptor.Builder()
+        .name("File Size")
+        .description("The size of the file that will be used")
+        .required(true)
+        .defaultValue("0 B")
+        .addValidator(StandardValidators.DATA_SIZE_VALIDATOR)
+        .build();
+    public static final PropertyDescriptor BATCH_SIZE = new PropertyDescriptor.Builder()
+        .name("Batch Size")
+        .description("The number of FlowFiles to be transferred in each invocation")
+        .required(true)
+        .defaultValue("1")
+        .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
+        .build();
+    public static final PropertyDescriptor CUSTOM_TEXT = new PropertyDescriptor.Builder()
+        .name("Text")
+        .description("If Data Format is text and if Unique FlowFiles is false, then this custom text will be used as content of the generated "
+            + "FlowFiles and the File Size will be ignored. Finally, if Expression Language is used, evaluation will be performed only once "
+            + "per batch of generated FlowFiles")
+        .required(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .build();
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+        .name("success")
+        .build();
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return Arrays.asList(FILE_SIZE, BATCH_SIZE, CUSTOM_TEXT);
+    }
+
+    @Override
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+        return new PropertyDescriptor.Builder()
+            .name(propertyDescriptorName)
+            .required(false)
+            .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING, true))
+            .addValidator(StandardValidators.ATTRIBUTE_KEY_PROPERTY_NAME_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .dynamic(true)
+            .build();
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return Collections.singleton(REL_SUCCESS);
+    }
+
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final int numFlowFiles = context.getProperty(BATCH_SIZE).asInteger();
+
+        for (int i=0; i < numFlowFiles; i++) {
+            final FlowFile flowFile = createFlowFile(context, session);
+            session.transfer(flowFile, REL_SUCCESS);
+        }
+    }
+
+    private FlowFile createFlowFile(final ProcessContext context, final ProcessSession session) {
+        FlowFile flowFile = session.create();
+
+        final Map<String, String> attributes = new HashMap<>();
+        context.getProperties().keySet().forEach(descriptor -> {
+            if (descriptor.isDynamic()) {
+                final String value = context.getProperty(descriptor).evaluateAttributeExpressions().getValue();
+                attributes.put(descriptor.getName(), value);
+            }
+        });
+
+        if (!attributes.isEmpty()) {
+            flowFile = session.putAllAttributes(flowFile, attributes);
+        }
+
+        final String customText = context.getProperty(CUSTOM_TEXT).evaluateAttributeExpressions().getValue();
+        if (customText == null) {
+            final int dataSize = context.getProperty(FILE_SIZE).asDataSize(DataUnit.B).intValue();
+            if (dataSize > 0L) {
+                final byte[] data = new byte[dataSize];
+                final Random random = new Random();
+                random.nextBytes(data);
+
+                flowFile = session.write(flowFile, out -> out.write(data));
+            }
+        } else {
+            flowFile = session.write(flowFile, out -> out.write(customText.getBytes(StandardCharsets.UTF_8)));
+        }
+
+        return flowFile;
+    }
+}
diff --git a/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/processors/tests/system/Sleep.java b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/processors/tests/system/Sleep.java
new file mode 100644
index 0000000..2a80113
--- /dev/null
+++ b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/processors/tests/system/Sleep.java
@@ -0,0 +1,141 @@
+/*
+ * 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.
+ */
+package org.apache.nifi.processors.tests.system;
+
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.PropertyDescriptor.Builder;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.cs.tests.system.SleepService;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+public class Sleep extends AbstractProcessor {
+    public static final PropertyDescriptor VALIDATE_SLEEP_TIME = new Builder()
+        .name("Validate Sleep Time")
+        .description("The amount of time to sleep during validation")
+        .required(false)
+        .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+        .defaultValue("0 sec")
+        .build();
+    public static final PropertyDescriptor ON_TRIGGER_SLEEP_TIME = new Builder()
+        .name("onTrigger Sleep Time")
+        .description("The amount of time to sleep during each trigger")
+        .required(false)
+        .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+        .defaultValue("0 sec")
+        .build();
+    public static final PropertyDescriptor ON_SCHEDULED_SLEEP_TIME = new Builder()
+        .name("@OnScheduled Sleep Time")
+        .description("The amount of time to sleep when scheduled")
+        .required(false)
+        .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+        .defaultValue("0 sec")
+        .build();
+    public static final PropertyDescriptor ON_STOPPED_SLEEP_TIME = new Builder()
+        .name("@OnStopped Sleep Time")
+        .description("The amount of time to sleep when stopped")
+        .required(false)
+        .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+        .defaultValue("0 sec")
+        .build();
+    static final PropertyDescriptor SLEEP_SERVICE = new Builder()
+        .name("Sleep Service")
+        .description("Controller Service that sleeps")
+        .required(false)
+        .identifiesControllerService(SleepService.class)
+        .build();
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+        .name("success")
+        .build();
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        final List<PropertyDescriptor> properties = new ArrayList<>();
+        properties.add(VALIDATE_SLEEP_TIME);
+        properties.add(ON_SCHEDULED_SLEEP_TIME);
+        properties.add(ON_TRIGGER_SLEEP_TIME);
+        properties.add(ON_STOPPED_SLEEP_TIME);
+        properties.add(SLEEP_SERVICE);
+        return properties;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return Collections.singleton(REL_SUCCESS);
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
+        final long sleepMillis = validationContext.getProperty(VALIDATE_SLEEP_TIME).asTimePeriod(TimeUnit.MILLISECONDS);
+        sleep(sleepMillis);
+
+        return Collections.emptyList();
+    }
+
+    private void sleep(final long millis) {
+        if (millis > 0L) {
+            try {
+                Thread.sleep(millis);
+            } catch (final InterruptedException ie) {
+                Thread.currentThread().interrupt();
+            }
+        }
+    }
+
+    @OnScheduled
+    public void onEnabled(final ProcessContext context) {
+        sleep(context.getProperty(ON_SCHEDULED_SLEEP_TIME).asTimePeriod(TimeUnit.MILLISECONDS));
+    }
+
+    @OnStopped
+    public void onDisabled(final ProcessContext context) {
+        sleep(context.getProperty(ON_STOPPED_SLEEP_TIME).asTimePeriod(TimeUnit.MILLISECONDS));
+    }
+
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final long sleepMillis = context.getProperty(ON_TRIGGER_SLEEP_TIME).asTimePeriod(TimeUnit.MILLISECONDS);
+        sleep(sleepMillis);
+
+        final SleepService service = context.getProperty(SLEEP_SERVICE).asControllerService(SleepService.class);
+        if (service != null) {
+            service.sleep();
+        }
+
+        FlowFile flowFile = session.get();
+        if (flowFile != null) {
+            session.transfer(flowFile, REL_SUCCESS);
+        }
+    }
+}
diff --git a/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/processors/tests/system/ValidateFileExists.java b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/processors/tests/system/ValidateFileExists.java
new file mode 100644
index 0000000..66bd95c
--- /dev/null
+++ b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/processors/tests/system/ValidateFileExists.java
@@ -0,0 +1,59 @@
+/*
+ * 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.
+ */
+package org.apache.nifi.processors.tests.system;
+
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+
+import java.util.Collections;
+import java.util.List;
+
+public class ValidateFileExists extends AbstractProcessor {
+
+    static final PropertyDescriptor FILE = new PropertyDescriptor.Builder()
+        .name("Filename")
+        .displayName("Filename")
+        .description("A file that should exist")
+        .required(true)
+        .addValidator(StandardValidators.FILE_EXISTS_VALIDATOR)
+        .build();
+
+    private static final Relationship REL_SUCCESS = new Relationship.Builder().name("success").build();
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return Collections.singletonList(FILE);
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        session.adjustCounter("Triggered", 1, true);
+
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        session.transfer(flowFile, REL_SUCCESS);
+    }
+}
diff --git a/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService
new file mode 100644
index 0000000..a76f5a0
--- /dev/null
+++ b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService
@@ -0,0 +1,16 @@
+# 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.
+
+org.apache.nifi.cs.tests.system.StandardSleepService
\ No newline at end of file
diff --git a/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
new file mode 100644
index 0000000..ffd9df7
--- /dev/null
+++ b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
@@ -0,0 +1,19 @@
+# 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.
+
+org.apache.nifi.processors.tests.system.CountEvents
+org.apache.nifi.processors.tests.system.GenerateFlowFile
+org.apache.nifi.processors.tests.system.Sleep
+org.apache.nifi.processors.tests.system.ValidateFileExists
\ No newline at end of file
diff --git a/nifi-system-tests/nifi-system-test-extensions-bundle/pom.xml b/nifi-system-tests/nifi-system-test-extensions-bundle/pom.xml
new file mode 100644
index 0000000..796bc55
--- /dev/null
+++ b/nifi-system-tests/nifi-system-test-extensions-bundle/pom.xml
@@ -0,0 +1,35 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <artifactId>nifi-system-tests</artifactId>
+        <groupId>org.apache.nifi</groupId>
+        <version>1.11.0-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+
+    <artifactId>nifi-system-test-extensions-bundle</artifactId>
+    <packaging>pom</packaging>
+
+    <modules>
+        <module>nifi-system-test-extensions</module>
+        <module>nifi-system-test-extensions-nar</module>
+    </modules>
+
+
+</project>
\ No newline at end of file
diff --git a/nifi-system-tests/nifi-system-test-suite/pom.xml b/nifi-system-tests/nifi-system-test-suite/pom.xml
new file mode 100644
index 0000000..821c705
--- /dev/null
+++ b/nifi-system-tests/nifi-system-test-suite/pom.xml
@@ -0,0 +1,207 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <artifactId>nifi-system-tests</artifactId>
+        <groupId>org.apache.nifi</groupId>
+        <version>1.11.0-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+
+    <artifactId>nifi-system-test-suite</artifactId>
+    <packaging>jar</packaging>
+
+    <build>
+        <plugins>
+            <plugin>
+                <artifactId>maven-assembly-plugin</artifactId>
+                <configuration>
+                    <finalName>nifi-lib</finalName>
+                    <attach>false</attach>
+                </configuration>
+                <executions>
+                    <execution>
+                        <id>make shared resource</id>
+                        <goals>
+                            <goal>single</goal>
+                        </goals>
+                        <phase>generate-test-resources</phase>
+                        <configuration>
+                            <archiverConfig>
+                                <defaultDirectoryMode>0775</defaultDirectoryMode>
+                                <directoryMode>0775</directoryMode>
+                                <fileMode>0664</fileMode>
+                            </archiverConfig>
+                            <descriptors>
+                                <descriptor>src/test/assembly/dependencies.xml</descriptor>
+                            </descriptors>
+                            <tarLongFileMode>posix</tarLongFileMode>
+                            <formats>
+                                <format>dir</format>
+                            </formats>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+        </plugins>
+    </build>
+
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-client-dto</artifactId>
+            <version>1.11.0-SNAPSHOT</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-toolkit-cli</artifactId>
+            <version>1.11.0-SNAPSHOT</version>
+            <scope>test</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.slf4j</groupId>
+                    <artifactId>slf4j-log4j12</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.glassfish.jersey.inject</groupId>
+            <artifactId>jersey-hk2</artifactId>
+            <version>2.27</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-simple</artifactId>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency> <!-- handling this explicitly  Must be in root lib -->
+            <groupId>javax.servlet</groupId>
+            <artifactId>javax.servlet-api</artifactId>
+            <scope>compile</scope>
+        </dependency>
+        <dependency> <!-- handling this explicitly  Must be in root lib -->
+            <groupId>org.eclipse.jetty.toolchain</groupId>
+            <artifactId>jetty-schemas</artifactId>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>jcl-over-slf4j</artifactId>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>jul-to-slf4j</artifactId>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-api</artifactId>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>ch.qos.logback</groupId>
+            <artifactId>logback-classic</artifactId>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+            <version>1.11.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-framework-api</artifactId>
+            <version>1.11.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-runtime</artifactId>
+            <version>1.11.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-bootstrap</artifactId>
+            <version>1.11.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-jetty-bundle</artifactId>
+            <version>1.11.0-SNAPSHOT</version>
+            <type>nar</type>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-framework-nar</artifactId>
+            <version>1.11.0-SNAPSHOT</version>
+            <type>nar</type>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-provenance-repository-nar</artifactId>
+            <version>1.11.0-SNAPSHOT</version>
+            <type>nar</type>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-standard-services-api-nar</artifactId>
+            <version>1.11.0-SNAPSHOT</version>
+            <type>nar</type>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-system-test-extensions-nar</artifactId>
+            <version>1.11.0-SNAPSHOT</version>
+            <type>nar</type>
+        </dependency>
+
+        <!-- dependencies for jaxb/activation/annotation for running NiFi on Java 11 -->
+        <!-- TODO: remove these once minimum Java version is 11 -->
+        <dependency>
+            <groupId>javax.xml.bind</groupId>
+            <artifactId>jaxb-api</artifactId>
+            <version>2.3.0</version>
+        </dependency>
+        <dependency>
+            <groupId>com.sun.xml.bind</groupId>
+            <artifactId>jaxb-impl</artifactId>
+            <version>2.3.0</version>
+        </dependency>
+        <dependency>
+            <groupId>com.sun.xml.bind</groupId>
+            <artifactId>jaxb-core</artifactId>
+            <version>2.3.0</version>
+        </dependency>
+        <dependency>
+            <groupId>javax.annotation</groupId>
+            <artifactId>javax.annotation-api</artifactId>
+            <version>1.3.2</version>
+        </dependency>
+        <dependency>
+            <groupId>javax.activation</groupId>
+            <artifactId>javax.activation-api</artifactId>
+            <version>1.2.0</version>
+        </dependency>
+
+    </dependencies>
+
+</project>
\ No newline at end of file
diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/assembly/dependencies.xml b/nifi-system-tests/nifi-system-test-suite/src/test/assembly/dependencies.xml
new file mode 100644
index 0000000..1c1b90a
--- /dev/null
+++ b/nifi-system-tests/nifi-system-test-suite/src/test/assembly/dependencies.xml
@@ -0,0 +1,80 @@
+<?xml version="1.0"?>
+<!--
+  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.
+-->
+<assembly>
+    <id>assembly</id>
+    <includeBaseDirectory>true</includeBaseDirectory>
+    <baseDirectory>./</baseDirectory>
+
+
+    <dependencySets>
+        <!-- Write out the bootstrap lib component to its own dir -->
+        <dependencySet>
+            <scope>runtime</scope>
+            <useProjectArtifact>false</useProjectArtifact>
+            <outputDirectory>lib/bootstrap</outputDirectory>
+            <directoryMode>0770</directoryMode>
+            <fileMode>0664</fileMode>
+            <useTransitiveFiltering>true</useTransitiveFiltering>
+            <includes>
+                <include>nifi-bootstrap</include>
+                <include>slf4j-api</include>
+                <include>nifi-api</include>
+            </includes>
+        </dependencySet>
+
+        <!-- Write out the bootstrap libs for java11 to its own dir -->
+        <!-- TODO: remove this dependency set once minimum Java version is 11 -->
+        <dependencySet>
+            <scope>runtime</scope>
+            <useProjectArtifact>false</useProjectArtifact>
+            <outputDirectory>lib/java11</outputDirectory>
+            <directoryMode>0770</directoryMode>
+            <fileMode>0664</fileMode>
+            <useTransitiveFiltering>true</useTransitiveFiltering>
+            <includes>
+                <include>javax.xml.bind:jaxb-api</include>
+                <include>com.sun.xml.bind:jaxb-impl</include>
+                <include>com.sun.xml.bind:jaxb-core</include>
+                <include>javax.activation:javax.activation-api</include>
+                <include>javax.annotation:javax.annotation-api</include>
+            </includes>
+        </dependencySet>
+
+        <!-- Write out all dependency artifacts to lib directory -->
+        <dependencySet>
+            <scope>runtime</scope>
+            <useProjectArtifact>false</useProjectArtifact>
+            <outputDirectory>lib</outputDirectory>
+            <directoryMode>0770</directoryMode>
+            <fileMode>0664</fileMode>
+            <useTransitiveFiltering>true</useTransitiveFiltering>
+            <excludes>
+                <exclude>nifi-bootstrap</exclude>
+                <exclude>nifi-resources</exclude>
+                <exclude>nifi-docs</exclude>
+
+                <!-- exclude jaxb/activation/annotation libs from lib, they'll be included in the java11 subdir -->
+                <!-- TODO: remove these once minimum Java version is 11 -->
+                <exclude>javax.xml.bind:jaxb-api</exclude>
+                <exclude>com.sun.xml.bind:jaxb-impl</exclude>
+                <exclude>com.sun.xml.bind:jaxb-core</exclude>
+                <exclude>javax.activation:javax.activation-api</exclude>
+                <exclude>javax.annotation:javax.annotation-api</exclude>
+            </excludes>
+        </dependencySet>
+    </dependencySets>
+
+</assembly>
diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/AggregateNiFiInstance.java b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/AggregateNiFiInstance.java
new file mode 100644
index 0000000..688e14a
--- /dev/null
+++ b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/AggregateNiFiInstance.java
@@ -0,0 +1,128 @@
+/*
+ * 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.
+ */
+package org.apache.nifi.tests.system;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+public class AggregateNiFiInstance implements NiFiInstance {
+    private final List<NiFiInstance> instances;
+
+    public AggregateNiFiInstance(final List<NiFiInstance> instances) {
+        this.instances = instances;
+    }
+
+    @Override
+    public void start() {
+        for (final NiFiInstance instance : instances) {
+            if (instance.isAutoStart()) {
+                instance.start();
+            }
+        }
+    }
+
+    @Override
+    public void createEnvironment() throws IOException {
+        for (final NiFiInstance instance : instances) {
+            instance.createEnvironment();
+        }
+    }
+
+    @Override
+    public void stop() {
+        Exception thrown = null;
+
+        // Shut down in the opposite order that they were brought up. We do this because only the first instance is going to be running ZooKeeper, and we don't
+        // want to kill that before the other instances are shutdown.
+        for (int i=instances.size() - 1; i >= 0; i--) {
+            final NiFiInstance instance = instances.get(i);
+
+            try {
+                instance.stop();
+            } catch (final Exception e) {
+                thrown = e;
+            }
+        }
+
+        if (thrown != null) {
+            throw (RuntimeException) thrown;
+        }
+    }
+
+
+    @Override
+    public boolean isClustered() {
+        return true;
+    }
+
+    @Override
+    public int getNumberOfNodes() {
+        return instances.size();
+    }
+
+    @Override
+    public int getNumberOfNodes(final boolean includeOnlyAutoStartInstances) {
+        if (includeOnlyAutoStartInstances) {
+            return (int) instances.stream()
+                .filter(NiFiInstance::isAutoStart)
+                .count();
+        }
+
+        return instances.size();
+    }
+
+    @Override
+    public NiFiInstance getNodeInstance(final int nodeIndex) {
+        if (nodeIndex < 1 || nodeIndex > instances.size()) {
+            throw new IllegalArgumentException("Node Index must be between 1 and " + instances.size() + "; invalid value given: " + nodeIndex);
+        }
+
+        return instances.get(nodeIndex - 1);
+    }
+
+    @Override
+    public Properties getProperties() {
+        return null;
+    }
+
+    @Override
+    public File getInstanceDirectory() {
+        return null;
+    }
+
+    @Override
+    public boolean isAutoStart() {
+        return true;
+    }
+
+    @Override
+    public void setProperty(final String propertyName, final String propertyValue) throws IOException {
+        for (final NiFiInstance instance : instances) {
+            instance.setProperty(propertyName, propertyValue);
+        }
+    }
+
+    @Override
+    public void setProperties(final Map<String, String> properties) throws IOException {
+        for (final NiFiInstance instance : instances) {
+            instance.setProperties(properties);
+        }
+    }
+}
diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/InstanceConfiguration.java b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/InstanceConfiguration.java
new file mode 100644
index 0000000..85a839c
--- /dev/null
+++ b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/InstanceConfiguration.java
@@ -0,0 +1,128 @@
+/*
+ * 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.
+ */
+package org.apache.nifi.tests.system;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+
+public class InstanceConfiguration {
+    private final File bootstrapConfigFile;
+    private final File instanceDirectory;
+    private final File flowXmlGz;
+    private final File stateDirectory;
+    private final boolean autoStart;
+
+    private InstanceConfiguration(Builder builder) {
+        this.bootstrapConfigFile = builder.bootstrapConfigFile;
+        this.instanceDirectory = builder.instanceDirectory;
+        this.flowXmlGz = builder.flowXmlGz;
+        this.stateDirectory = builder.stateDirectory;
+        this.autoStart = builder.autoStart;
+    }
+
+    public File getBootstrapConfigFile() {
+        return bootstrapConfigFile;
+    }
+
+    public File getInstanceDirectory() {
+        return instanceDirectory;
+    }
+
+    public File getFlowXmlGz() {
+        return flowXmlGz;
+    }
+
+    public File getStateDirectory() {
+        return stateDirectory;
+    }
+
+    public boolean isAutoStart() {
+        return autoStart;
+    }
+
+    public static class Builder {
+        private File bootstrapConfigFile;
+        private File instanceDirectory;
+        private File flowXmlGz;
+        private File stateDirectory;
+        private boolean autoStart = true;
+
+        public Builder bootstrapConfig(final File configFile) {
+            if (!configFile.exists()) {
+                throw new RuntimeException(new FileNotFoundException(configFile.getAbsolutePath()));
+            }
+
+            this.bootstrapConfigFile = configFile;
+            return this;
+        }
+
+        public Builder bootstrapConfig(final String configFilename) {
+            return bootstrapConfig(new File(configFilename));
+        }
+
+        public Builder instanceDirectory(final File instanceDir) {
+            this.instanceDirectory = instanceDir;
+            return this;
+        }
+
+        public Builder instanceDirectory(final String instanceDirName) {
+            return instanceDirectory(new File(instanceDirName));
+        }
+
+        public Builder flowXml(final File flowXml) {
+            this.flowXmlGz = flowXml;
+            return this;
+        }
+
+        public Builder flowXml(final String flowXmlFilename) {
+            return flowXml(new File(flowXmlFilename));
+        }
+
+        public Builder stateDirectory(final File stateDirectory) {
+            if (!stateDirectory.exists()) {
+                throw new RuntimeException(new FileNotFoundException(stateDirectory.getAbsolutePath()));
+            }
+
+            if (!stateDirectory.isDirectory()) {
+                throw new RuntimeException("Specified State Directory " + stateDirectory.getAbsolutePath() + " is not a directory");
+            }
+
+            this.stateDirectory = stateDirectory;
+            return this;
+        }
+
+        public Builder stateDirectory(final String stateDirectoryName) {
+            return stateDirectory(new File(stateDirectoryName));
+        }
+
+        public Builder autoStart(boolean autoStart) {
+            this.autoStart = autoStart;
+            return this;
+        }
+
+        public InstanceConfiguration build() {
+            if (instanceDirectory == null) {
+                throw new IllegalStateException("Instance Directory has not been specified");
+            }
+            if (bootstrapConfigFile == null) {
+                throw new IllegalStateException("Bootstrap Config File has not been specified");
+            }
+
+            return new InstanceConfiguration(this);
+        }
+    }
+}
diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/NiFiClientUtil.java b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/NiFiClientUtil.java
new file mode 100644
index 0000000..21f04f3
--- /dev/null
+++ b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/NiFiClientUtil.java
@@ -0,0 +1,662 @@
+/*
+ * 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.
+ */
+package org.apache.nifi.tests.system;
+
+import org.apache.nifi.cluster.coordination.node.NodeConnectionState;
+import org.apache.nifi.controller.queue.LoadBalanceCompression;
+import org.apache.nifi.controller.queue.LoadBalanceStrategy;
+import org.apache.nifi.controller.queue.QueueSize;
+import org.apache.nifi.remote.protocol.SiteToSiteTransportProtocol;
+import org.apache.nifi.scheduling.ExecutionNode;
+import org.apache.nifi.toolkit.cli.impl.client.nifi.ConnectionClient;
+import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClient;
+import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientException;
+import org.apache.nifi.web.api.dto.BundleDTO;
+import org.apache.nifi.web.api.dto.ConnectableDTO;
+import org.apache.nifi.web.api.dto.ConnectionDTO;
+import org.apache.nifi.web.api.dto.ControllerServiceDTO;
+import org.apache.nifi.web.api.dto.CounterDTO;
+import org.apache.nifi.web.api.dto.CountersSnapshotDTO;
+import org.apache.nifi.web.api.dto.FlowSnippetDTO;
+import org.apache.nifi.web.api.dto.NodeDTO;
+import org.apache.nifi.web.api.dto.ParameterContextDTO;
+import org.apache.nifi.web.api.dto.ParameterContextReferenceDTO;
+import org.apache.nifi.web.api.dto.ParameterDTO;
+import org.apache.nifi.web.api.dto.PortDTO;
+import org.apache.nifi.web.api.dto.ProcessGroupDTO;
+import org.apache.nifi.web.api.dto.ProcessorConfigDTO;
+import org.apache.nifi.web.api.dto.ProcessorDTO;
+import org.apache.nifi.web.api.dto.RemoteProcessGroupDTO;
+import org.apache.nifi.web.api.dto.RevisionDTO;
+import org.apache.nifi.web.api.dto.flow.FlowDTO;
+import org.apache.nifi.web.api.dto.flow.ProcessGroupFlowDTO;
+import org.apache.nifi.web.api.dto.status.ConnectionStatusSnapshotDTO;
+import org.apache.nifi.web.api.entity.ActivateControllerServicesEntity;
+import org.apache.nifi.web.api.entity.ConnectionEntity;
+import org.apache.nifi.web.api.entity.ConnectionStatusEntity;
+import org.apache.nifi.web.api.entity.ControllerServiceEntity;
+import org.apache.nifi.web.api.entity.ControllerServiceRunStatusEntity;
+import org.apache.nifi.web.api.entity.ControllerServicesEntity;
+import org.apache.nifi.web.api.entity.CountersEntity;
+import org.apache.nifi.web.api.entity.DropRequestEntity;
+import org.apache.nifi.web.api.entity.NodeEntity;
+import org.apache.nifi.web.api.entity.ParameterContextEntity;
+import org.apache.nifi.web.api.entity.ParameterContextReferenceEntity;
+import org.apache.nifi.web.api.entity.ParameterContextUpdateRequestEntity;
+import org.apache.nifi.web.api.entity.ParameterEntity;
+import org.apache.nifi.web.api.entity.PortEntity;
+import org.apache.nifi.web.api.entity.ProcessGroupEntity;
+import org.apache.nifi.web.api.entity.ProcessGroupFlowEntity;
+import org.apache.nifi.web.api.entity.ProcessorEntity;
+import org.apache.nifi.web.api.entity.RemoteProcessGroupEntity;
+import org.apache.nifi.web.api.entity.ScheduleComponentsEntity;
+import org.junit.Assert;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+public class NiFiClientUtil {
+    private final NiFiClient nifiClient;
+    private final String nifiVersion;
+
+    public NiFiClientUtil(final NiFiClient client, final String nifiVersion) {
+        this.nifiClient = client;
+        this.nifiVersion = nifiVersion;
+    }
+
+    public ProcessorEntity createProcessor(final String simpleTypeName) throws NiFiClientException, IOException {
+        return createProcessor(NiFiSystemIT.TEST_PROCESSORS_PACKAGE + "." + simpleTypeName, NiFiSystemIT.NIFI_GROUP_ID, NiFiSystemIT.TEST_EXTENSIONS_ARTIFACT_ID, nifiVersion);
+    }
+
+    public ProcessorEntity createProcessor(final String type, final String groupId, final String artifactId, final String version) throws NiFiClientException, IOException {
+        final ProcessorDTO dto = new ProcessorDTO();
+        dto.setType(type);
+
+        final BundleDTO bundle = new BundleDTO();
+        bundle.setGroup(groupId);
+        bundle.setArtifact(artifactId);
+        bundle.setVersion(version);
+        dto.setBundle(bundle);
+
+        final ProcessorEntity entity = new ProcessorEntity();
+        entity.setComponent(dto);
+        entity.setRevision(createNewRevision());
+
+        return nifiClient.getProcessorClient().createProcessor("root", entity);
+    }
+
+    public ControllerServiceEntity createControllerService(final String simpleTypeName) throws NiFiClientException, IOException {
+        return createControllerService(NiFiSystemIT.TEST_CS_PACKAGE + "." + simpleTypeName, "root", NiFiSystemIT.NIFI_GROUP_ID, NiFiSystemIT.TEST_EXTENSIONS_ARTIFACT_ID, nifiVersion);
+    }
+
+    public ControllerServiceEntity createControllerService(final String type, final String processGroupId, final String bundleGroupId, final String artifactId, final String version)
+                throws NiFiClientException, IOException {
+        final ControllerServiceDTO dto = new ControllerServiceDTO();
+        dto.setType(type);
+
+        final BundleDTO bundle = new BundleDTO();
+        bundle.setGroup(bundleGroupId);
+        bundle.setArtifact(artifactId);
+        bundle.setVersion(version);
+        dto.setBundle(bundle);
+
+        final ControllerServiceEntity entity = new ControllerServiceEntity();
+        entity.setComponent(dto);
+        entity.setRevision(createNewRevision());
+
+        return nifiClient.getControllerServicesClient().createControllerService(processGroupId, entity);
+    }
+
+    public ParameterEntity createParameterEntity(final String name, final String description, final boolean sensitive, final String value) {
+        final ParameterDTO dto = new ParameterDTO();
+        dto.setName(name);
+        dto.setDescription(description);
+        dto.setSensitive(sensitive);
+        dto.setValue(value);
+
+        final ParameterEntity entity = new ParameterEntity();
+        entity.setParameter(dto);
+        return entity;
+    }
+
+    public ParameterContextEntity createParameterContextEntity(final String name, final String description, final Set<ParameterEntity> parameters) {
+        final ParameterContextDTO contextDto = new ParameterContextDTO();
+        contextDto.setName(name);
+        contextDto.setDescription(description);
+        contextDto.setParameters(parameters);
+
+        final ParameterContextEntity entity = new ParameterContextEntity();
+        entity.setComponent(contextDto);
+        entity.setRevision(createNewRevision());
+
+        return entity;
+    }
+
+    private RevisionDTO createNewRevision() {
+        final RevisionDTO revisionDto = new RevisionDTO();
+        revisionDto.setClientId(getClass().getName());
+        revisionDto.setVersion(0L);
+        return revisionDto;
+    }
+
+    private ParameterContextReferenceEntity createReferenceEntity(final String id) {
+        return createReferenceEntity(id, null);
+    }
+
+    private ParameterContextReferenceEntity createReferenceEntity(final String id, final String name) {
+        final ParameterContextReferenceDTO referenceDto = new ParameterContextReferenceDTO();
+        referenceDto.setId(id);
+        referenceDto.setName(name);
+
+        final ParameterContextReferenceEntity referenceEntity = new ParameterContextReferenceEntity();
+        referenceEntity.setId(id);
+        referenceEntity.setComponent(referenceDto);
+
+        return referenceEntity;
+    }
+
+    public ProcessGroupEntity setParameterContext(final String groupId, final ParameterContextEntity parameterContext) throws NiFiClientException, IOException {
+        final ProcessGroupEntity processGroup = nifiClient.getProcessGroupClient().getProcessGroup(groupId);
+        processGroup.getComponent().setParameterContext(createReferenceEntity(parameterContext.getId()));
+        return nifiClient.getProcessGroupClient().updateProcessGroup(processGroup);
+    }
+
+    public ParameterContextEntity createParameterContext(final String contextName, final String parameterName, final String parameterValue) throws NiFiClientException, IOException {
+        return createParameterContext(contextName, Collections.singletonMap(parameterName, parameterValue));
+    }
+
+    public ParameterContextEntity createParameterContext(final String contextName, final Map<String, String> parameters) throws NiFiClientException, IOException {
+        final Set<ParameterEntity> parameterEntities = new HashSet<>();
+        parameters.forEach((paramName, paramValue) -> parameterEntities.add(createParameterEntity(paramName, null, false, paramValue)));
+
+        final ParameterContextEntity contextEntity = createParameterContextEntity(contextName, null, parameterEntities);
+        final ParameterContextEntity createdContextEntity = nifiClient.getParamContextClient().createParamContext(contextEntity);
+        return createdContextEntity;
+    }
+
+    public ParameterContextUpdateRequestEntity updateParameterContext(final ParameterContextEntity existingEntity, final String paramName, final String paramValue)
+        throws NiFiClientException, IOException {
+        return updateParameterContext(existingEntity, Collections.singletonMap(paramName, paramValue));
+    }
+
+    public ParameterContextUpdateRequestEntity updateParameterContext(final ParameterContextEntity existingEntity, final Map<String, String> parameters) throws NiFiClientException, IOException {
+        final Set<ParameterEntity> parameterEntities = new HashSet<>();
+        parameters.forEach((paramName, paramValue) -> parameterEntities.add(createParameterEntity(paramName, null, false, paramValue)));
+        existingEntity.getComponent().setParameters(parameterEntities);
+
+        return nifiClient.getParamContextClient().updateParamContext(existingEntity);
+    }
+
+    public void waitForParameterContextRequestToComplete(final String contextId, final String requestId) throws NiFiClientException, IOException, InterruptedException {
+        while (true) {
+            final ParameterContextUpdateRequestEntity entity = nifiClient.getParamContextClient().getParamContextUpdateRequest(contextId, requestId);
+            if (entity.getRequest().isComplete()) {
+                return;
+            }
+
+            Thread.sleep(100L);
+        }
+    }
+
+
+    public ProcessorEntity updateProcessorExecutionNode(final ProcessorEntity currentEntity, final ExecutionNode executionNode) throws NiFiClientException, IOException {
+        final ProcessorConfigDTO config = new ProcessorConfigDTO();
+        config.setExecutionNode(executionNode.name());
+        return updateProcessorConfig(currentEntity, config);
+    }
+
+    public ProcessorEntity updateProcessorProperties(final ProcessorEntity currentEntity, final Map<String, String> properties) throws NiFiClientException, IOException {
+        final ProcessorConfigDTO config = new ProcessorConfigDTO();
+        config.setProperties(properties);
+        return updateProcessorConfig(currentEntity, config);
+    }
+
+    public ProcessorEntity updateProcessorSchedulingPeriod(final ProcessorEntity currentEntity, final String schedulingPeriod) throws NiFiClientException, IOException {
+        final ProcessorConfigDTO config = new ProcessorConfigDTO();
+        config.setSchedulingPeriod(schedulingPeriod);
+        return updateProcessorConfig(currentEntity, config);
+    }
+
+    public ProcessorEntity updateProcessorConfig(final ProcessorEntity currentEntity, final ProcessorConfigDTO config) throws NiFiClientException, IOException {
+        final ProcessorDTO processorDto = new ProcessorDTO();
+        processorDto.setConfig(config);
+        processorDto.setId(currentEntity.getId());
+
+        final ProcessorEntity updatedEntity = new ProcessorEntity();
+        updatedEntity.setRevision(currentEntity.getRevision());
+        updatedEntity.setComponent(processorDto);
+        updatedEntity.setId(currentEntity.getId());
+
+        return nifiClient.getProcessorClient().updateProcessor(updatedEntity);
+    }
+
+    public ProcessorEntity setAutoTerminatedRelationships(final ProcessorEntity currentEntity, final String autoTerminatedRelationship) throws NiFiClientException, IOException {
+        return setAutoTerminatedRelationships(currentEntity, Collections.singleton(autoTerminatedRelationship));
+    }
+
+    public ProcessorEntity setAutoTerminatedRelationships(final ProcessorEntity currentEntity, final Set<String> autoTerminatedRelationships) throws NiFiClientException, IOException {
+        final ProcessorConfigDTO config = new ProcessorConfigDTO();
+        config.setAutoTerminatedRelationships(autoTerminatedRelationships);
+        return updateProcessorConfig(currentEntity, config);
+    }
+
+    public void waitForValidProcessor(String id) throws InterruptedException, IOException, NiFiClientException {
+        waitForValidStatus(id, ProcessorDTO.VALID);
+    }
+
+    public void waitForInvalidProcessor(String id) throws NiFiClientException, IOException, InterruptedException {
+        waitForValidStatus(id, ProcessorDTO.INVALID);
+    }
+
+    public void waitForValidStatus(final String processorId, final String expectedStatus) throws NiFiClientException, IOException, InterruptedException {
+        while (true) {
+            final ProcessorEntity entity = nifiClient.getProcessorClient().getProcessor(processorId);
+            final String validationStatus = entity.getComponent().getValidationStatus();
+            if (expectedStatus.equals(validationStatus)) {
+                return;
+            }
+
+            if ("Validating".equals(validationStatus)) {
+                Thread.sleep(100L);
+                continue;
+            }
+
+            Thread.sleep(100L);
+        }
+    }
+
+    public void waitForRunningProcessor(final String processorId) throws InterruptedException, IOException, NiFiClientException {
+        waitForProcessorState(processorId, "RUNNING");
+    }
+
+    public void waitForStoppedProcessor(final String processorId) throws InterruptedException, IOException, NiFiClientException {
+        waitForProcessorState(processorId, "STOPPED");
+    }
+
+    public void waitForProcessorState(final String processorId, final String expectedState) throws NiFiClientException, IOException, InterruptedException {
+        while (true) {
+            final ProcessorEntity entity = nifiClient.getProcessorClient().getProcessor(processorId);
+            final String state = entity.getComponent().getState();
+            if (!expectedState.equals(state)) {
+                Thread.sleep(10L);
+                continue;
+            }
+
+            if ("RUNNING".equals(expectedState)) {
+                return;
+            }
+
+            if (entity.getStatus().getAggregateSnapshot().getActiveThreadCount() == 0) {
+                return;
+            }
+
+            Thread.sleep(10L);
+        }
+    }
+
+    public ControllerServiceEntity enableControllerService(final ControllerServiceEntity entity) throws NiFiClientException, IOException {
+        final ControllerServiceRunStatusEntity runStatusEntity = new ControllerServiceRunStatusEntity();
+        runStatusEntity.setState("ENABLED");
+        runStatusEntity.setRevision(entity.getRevision());
+
+        return nifiClient.getControllerServicesClient().activateControllerService(entity.getId(), runStatusEntity);
+    }
+
+    public ControllerServiceEntity disableControllerService(final ControllerServiceEntity entity) throws NiFiClientException, IOException {
+        final ControllerServiceRunStatusEntity runStatusEntity = new ControllerServiceRunStatusEntity();
+        runStatusEntity.setState("DISABLED");
+        runStatusEntity.setRevision(entity.getRevision());
+
+        return nifiClient.getControllerServicesClient().activateControllerService(entity.getId(), runStatusEntity);
+    }
+
+    public Map<String, Long> waitForCounter(final String context, final String counterName, final long expectedValue) throws NiFiClientException, IOException, InterruptedException {
+        Map<String, Long> counterValues = getCountersAsMap(context);
+        while (true) {
+            final Long counterValue = counterValues.get(counterName);
+            if (counterValue != null && counterValue.longValue() == expectedValue) {
+                return counterValues;
+            }
+
+            Thread.sleep(10L);
+            counterValues = getCountersAsMap(context);
+        }
+    }
+
+
+    public Map<String, Long> getCountersAsMap(final String processorId) throws NiFiClientException, IOException {
+        final CountersEntity firstCountersEntity = nifiClient.getCountersClient().getCounters();
+        final CountersSnapshotDTO firstCounters = firstCountersEntity.getCounters().getAggregateSnapshot();
+        final Map<String, Long> counterValues = firstCounters.getCounters().stream()
+            .filter(dto -> dto.getContext().contains(processorId))
+            .collect(Collectors.toMap(CounterDTO::getName, CounterDTO::getValueCount));
+
+        return counterValues;
+    }
+
+    public ScheduleComponentsEntity stopProcessGroupComponents(final String groupId) throws NiFiClientException, IOException {
+        final ScheduleComponentsEntity scheduleComponentsEntity = new ScheduleComponentsEntity();
+        scheduleComponentsEntity.setId("root");
+        scheduleComponentsEntity.setState("STOPPED");
+        final ScheduleComponentsEntity scheduleEntity = nifiClient.getFlowClient().scheduleProcessGroupComponents("root", scheduleComponentsEntity);
+        waitForProcessorsStopped("root");
+
+        return scheduleEntity;
+    }
+
+    private void waitForProcessorsStopped(final String groupId) throws IOException, NiFiClientException {
+        final ProcessGroupFlowEntity rootGroup = nifiClient.getFlowClient().getProcessGroup(groupId);
+        final FlowDTO rootFlowDTO = rootGroup.getProcessGroupFlow().getFlow();
+        for (final ProcessorEntity processor : rootFlowDTO.getProcessors()) {
+            try {
+                waitForStoppedProcessor(processor.getId());
+            } catch (InterruptedException e) {
+                Thread.currentThread().interrupt();
+                throw new NiFiClientException("Interrupted while waiting for Processor with ID " + processor.getId() + " to stop");
+            }
+        }
+
+        for (final ProcessGroupEntity group : rootFlowDTO.getProcessGroups()) {
+            waitForProcessorsStopped(group.getComponent());
+        }
+    }
+
+    private void waitForProcessorsStopped(final ProcessGroupDTO group) throws IOException, NiFiClientException {
+        final FlowSnippetDTO groupContents = group.getContents();
+        for (final ProcessorDTO processor : groupContents.getProcessors()) {
+            try {
+                waitForStoppedProcessor(processor.getId());
+            } catch (InterruptedException e) {
+                Thread.currentThread().interrupt();
+                throw new NiFiClientException("Interrupted while waiting for Processor with ID " + processor.getId() + " to stop");
+            }
+        }
+
+        for (final ProcessGroupDTO child : groupContents.getProcessGroups()) {
+            waitForProcessorsStopped(child);
+        }
+    }
+
+    public void stopTransmitting(final String processGroupId) throws NiFiClientException, IOException {
+        final ProcessGroupFlowEntity rootGroup = nifiClient.getFlowClient().getProcessGroup(processGroupId);
+        final FlowDTO flowDto = rootGroup.getProcessGroupFlow().getFlow();
+
+        for (final RemoteProcessGroupEntity rpg : flowDto.getRemoteProcessGroups()) {
+            nifiClient.getRemoteProcessGroupClient().stopTransmitting(rpg);
+        }
+
+        for (final ProcessGroupEntity childGroup : flowDto.getProcessGroups()) {
+            stopTransmitting(childGroup.getId());
+        }
+    }
+
+    public ActivateControllerServicesEntity disableControllerServices(final String groupId) throws NiFiClientException, IOException {
+        final ActivateControllerServicesEntity activateControllerServicesEntity = new ActivateControllerServicesEntity();
+        activateControllerServicesEntity.setId(groupId);
+        activateControllerServicesEntity.setState(ActivateControllerServicesEntity.STATE_DISABLED);
+
+        final ActivateControllerServicesEntity activateControllerServices = nifiClient.getFlowClient().activateControllerServices(activateControllerServicesEntity);
+        waitForControllerSerivcesDisabled(groupId);
+
+        return activateControllerServices;
+    }
+
+    public void waitForControllerSerivcesDisabled(final String groupId, final String... serviceIdsOfInterest) throws NiFiClientException, IOException {
+        waitForControllerServiceState(groupId, "DISABLED", Arrays.asList(serviceIdsOfInterest));
+    }
+
+    public void waitForControllerSerivcesEnabled(final String groupId, final String... serviceIdsOfInterest) throws NiFiClientException, IOException {
+        waitForControllerServiceState(groupId, "ENABLED", Arrays.asList(serviceIdsOfInterest));
+    }
+
+    public void waitForControllerServiceState(final String groupId, final String desiredState, final Collection<String> serviceIdsOfInterest) throws NiFiClientException, IOException {
+        while (true) {
+            final List<ControllerServiceEntity> nonDisabledServices = getControllerServicesNotInState(groupId, desiredState, serviceIdsOfInterest);
+            if (nonDisabledServices.isEmpty()) {
+                System.out.println(String.format("All Controller Services in Process Group %s now have desired state of %s", groupId, desiredState));
+                return;
+            }
+
+            final ControllerServiceEntity entity = nonDisabledServices.get(0);
+            System.out.println(String.format("Controller Service with ID %s and type %s has a State of %s while waiting for state of %s; will wait 500 millis and check again", entity.getId(),
+                entity.getComponent().getType(), entity.getComponent().getState(), desiredState));
+
+            try {
+                Thread.sleep(500L);
+            } catch (final Exception e) {
+                e.printStackTrace();
+                Assert.fail(e.toString());
+            }
+        }
+    }
+
+    public List<ControllerServiceEntity> getControllerServicesNotInState(final String groupId, final String desiredState, final Collection<String> serviceIds) throws NiFiClientException, IOException {
+        final ControllerServicesEntity servicesEntity = nifiClient.getFlowClient().getControllerServices(groupId);
+
+        return servicesEntity.getControllerServices().stream()
+            .filter(svc -> serviceIds == null || serviceIds.isEmpty() || serviceIds.contains(svc.getId()))
+            .filter(svc -> !desiredState.equals(svc.getStatus().getRunStatus()))
+            .collect(Collectors.toList());
+    }
+
+    public void deleteAll(final String groupId) throws NiFiClientException, IOException {
+        final ProcessGroupFlowEntity rootFlowEntity = nifiClient.getFlowClient().getProcessGroup(groupId);
+        final ProcessGroupFlowDTO groupFlowDto = rootFlowEntity.getProcessGroupFlow();
+        final FlowDTO flowDto = groupFlowDto.getFlow();
+
+        // Delete all connections
+        for (final ConnectionEntity connectionEntity : flowDto.getConnections()) {
+            final ConnectionStatusEntity status = nifiClient.getFlowClient().getConnectionStatus(connectionEntity.getId(), false);
+            if (status.getConnectionStatus().getAggregateSnapshot().getFlowFilesQueued() > 0) {
+                emptyQueue(connectionEntity.getId());
+            }
+
+            nifiClient.getConnectionClient().deleteConnection(connectionEntity);
+        }
+
+        // Delete all processors
+        for (final ProcessorEntity processorEntity : flowDto.getProcessors()) {
+            nifiClient.getProcessorClient().deleteProcessor(processorEntity);
+        }
+
+        // Delete all Controller Services
+        final ControllerServicesEntity servicesEntity = nifiClient.getFlowClient().getControllerServices(groupId);
+        for (final ControllerServiceEntity serviceEntity : servicesEntity.getControllerServices()) {
+            nifiClient.getControllerServicesClient().deleteControllerService(serviceEntity);
+        }
+
+        // Delete all RPG's
+        for (final RemoteProcessGroupEntity rpgEntity : flowDto.getRemoteProcessGroups()) {
+            nifiClient.getRemoteProcessGroupClient().deleteRemoteProcessGroup(rpgEntity);
+        }
+
+        // Delete all Input Ports
+        for (final PortEntity port : flowDto.getInputPorts()) {
+            nifiClient.getInputPortClient().deleteInputPort(port);
+        }
+
+        // Delete all Output Ports
+        for (final PortEntity port : flowDto.getOutputPorts()) {
+            nifiClient.getOutputPortClient().deleteOutputPort(port);
+        }
+
+        // Recurse
+        for (final ProcessGroupEntity childGroupEntity : flowDto.getProcessGroups()) {
+            deleteAll(childGroupEntity.getId());
+        }
+    }
+
+    public ConnectionEntity createConnection(final ProcessorEntity source, final ProcessorEntity destination, final String relationship) throws NiFiClientException, IOException {
+        return createConnection(source, destination, Collections.singleton(relationship));
+    }
+
+    public ConnectionEntity createConnection(final ConnectableDTO source, final ConnectableDTO destination, final String relationship) throws NiFiClientException, IOException {
+        return createConnection(source, destination, Collections.singleton(relationship));
+    }
+
+    public ConnectionEntity createConnection(final ProcessorEntity source, final ProcessorEntity destination, final Set<String> relationships) throws NiFiClientException, IOException {
+        return createConnection(createConnectableDTO(source), createConnectableDTO(destination), relationships);
+    }
+
+    public ConnectionEntity createConnection(final ConnectableDTO source, final ConnectableDTO destination, final Set<String> relationships) throws NiFiClientException, IOException {
+        final ConnectionDTO connectionDto = new ConnectionDTO();
+        connectionDto.setSelectedRelationships(relationships);
+        connectionDto.setDestination(destination);
+        connectionDto.setSource(source);
+        connectionDto.setParentGroupId(source.getGroupId());
+
+        final ConnectionEntity connectionEntity = new ConnectionEntity();
+        connectionEntity.setComponent(connectionDto);
+
+        connectionEntity.setDestinationGroupId(destination.getGroupId());
+        connectionEntity.setDestinationId(destination.getId());
+        connectionEntity.setDestinationType("PROCESSOR");
+
+        connectionEntity.setSourceGroupId(source.getGroupId());
+        connectionEntity.setSourceId(source.getId());
+        connectionEntity.setDestinationType("PROCESSOR");
+
+        connectionEntity.setRevision(createNewRevision());
+
+        return nifiClient.getConnectionClient().createConnection(source.getGroupId(), connectionEntity);
+    }
+
+    public ConnectableDTO createConnectableDTO(final ProcessorEntity processor) {
+        final ConnectableDTO dto = new ConnectableDTO();
+        dto.setGroupId(processor.getComponent().getParentGroupId());
+        dto.setId(processor.getId());
+        dto.setName(processor.getComponent().getName());
+        dto.setRunning("RUNNING".equalsIgnoreCase(processor.getComponent().getState()));
+        dto.setType("PROCESSOR");
+
+        return dto;
+    }
+
+    public ConnectableDTO createConnectableDTO(final PortEntity port) {
+        final ConnectableDTO dto = new ConnectableDTO();
+        dto.setGroupId(port.getComponent().getParentGroupId());
+        dto.setId(port.getId());
+        dto.setName(port.getComponent().getName());
+        dto.setRunning("RUNNING".equalsIgnoreCase(port.getComponent().getState()));
+        dto.setType(port.getPortType());
+
+        return dto;
+    }
+
+    public QueueSize getQueueSize(String connectionId) {
+        try {
+            final ConnectionStatusEntity statusEntity = nifiClient.getFlowClient().getConnectionStatus(connectionId, false);
+            final ConnectionStatusSnapshotDTO snapshotDto = statusEntity.getConnectionStatus().getAggregateSnapshot();
+            return new QueueSize(snapshotDto.getFlowFilesQueued(), snapshotDto.getBytesQueued());
+        } catch (Exception e) {
+            throw new RuntimeException("Failed to obtain queue size for connection with ID " + connectionId, e);
+        }
+    }
+
+    public ConnectionEntity updateConnectionLoadBalancing(final ConnectionEntity connectionEntity, final LoadBalanceStrategy strategy, final LoadBalanceCompression compression,
+                                                          final String loadBalanceAttribute) throws NiFiClientException, IOException {
+        final ConnectionDTO connectionDto = new ConnectionDTO();
+        connectionDto.setLoadBalancePartitionAttribute(loadBalanceAttribute);
+        connectionDto.setLoadBalanceStrategy(strategy.name());
+        connectionDto.setLoadBalanceCompression(compression.name());
+        connectionDto.setId(connectionEntity.getId());
+
+        final ConnectionEntity updatedEntity = new ConnectionEntity();
+        updatedEntity.setComponent(connectionDto);
+        updatedEntity.setId(connectionEntity.getId());
+        updatedEntity.setRevision(connectionEntity.getRevision());
+
+        return nifiClient.getConnectionClient().updateConnection(updatedEntity);
+    }
+
+    public DropRequestEntity emptyQueue(final String connectionId) throws NiFiClientException, IOException {
+        final ConnectionClient connectionClient = nifiClient.getConnectionClient();
+
+        DropRequestEntity requestEntity = connectionClient.emptyQueue(connectionId);
+        try {
+            while (requestEntity.getDropRequest().getPercentCompleted() < 100) {
+                try {
+                    Thread.sleep(10L);
+                } catch (final InterruptedException ie) {
+                    Thread.currentThread().interrupt();
+                    return null;
+                }
+
+                requestEntity = connectionClient.getDropRequest(connectionId, requestEntity.getDropRequest().getId());
+            }
+        } finally {
+            requestEntity = connectionClient.deleteDropRequest(connectionId, requestEntity.getDropRequest().getId());
+        }
+
+        return requestEntity;
+    }
+
+    public RemoteProcessGroupEntity createRPG(final String parentGroupId, final SiteToSiteTransportProtocol transportProtocol) throws NiFiClientException, IOException {
+        final RemoteProcessGroupDTO component = new RemoteProcessGroupDTO();
+        component.setTargetUri("http://localhost:5671");
+        component.setName(component.getTargetUri());
+        component.setTransportProtocol(transportProtocol.name());
+
+        final RemoteProcessGroupEntity entity = new RemoteProcessGroupEntity();
+        entity.setComponent(component);
+        entity.setRevision(createNewRevision());
+
+        return nifiClient.getRemoteProcessGroupClient().createRemoteProcessGroup(parentGroupId, entity);
+    }
+
+    public PortEntity createRemoteInputPort(final String parentGroupId, final String portName) throws NiFiClientException, IOException {
+        final PortDTO component = new PortDTO();
+        component.setName(portName);
+        component.setAllowRemoteAccess(true);
+
+        final PortEntity entity = new PortEntity();
+        entity.setComponent(component);
+        entity.setRevision(createNewRevision());
+
+        return nifiClient.getInputPortClient().createInputPort(parentGroupId, entity);
+    }
+
+    public NodeEntity disconnectNode(final String nodeId) throws NiFiClientException, IOException {
+        return updateNodeState(nodeId, NodeConnectionState.DISCONNECTING.name());
+    }
+
+    public NodeEntity connectNode(final String nodeId) throws NiFiClientException, IOException {
+        return updateNodeState(nodeId, NodeConnectionState.CONNECTING.name());
+    }
+
+    public NodeEntity offloadNode(final String nodeId) throws NiFiClientException, IOException {
+        return updateNodeState(nodeId, NodeConnectionState.OFFLOADING.name());
+    }
+
+    private NodeEntity updateNodeState(final String nodeId, final String state) throws NiFiClientException, IOException {
+        final NodeDTO nodeDto = new NodeDTO();
+        nodeDto.setNodeId(nodeId);
+        nodeDto.setStatus(state);
+
+        final NodeEntity nodeEntity = new NodeEntity();
+        nodeEntity.setNode(nodeDto);
+
+        return nifiClient.getControllerClient().disconnectNode(nodeId, nodeEntity);
+    }
+}
diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/NiFiInstance.java b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/NiFiInstance.java
new file mode 100644
index 0000000..d24cb7b
--- /dev/null
+++ b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/NiFiInstance.java
@@ -0,0 +1,103 @@
+/*
+ * 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.
+ */
+package org.apache.nifi.tests.system;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Map;
+import java.util.Properties;
+
+public interface NiFiInstance {
+
+    /**
+     * Sets up the environment for the instance to run in
+     */
+    void createEnvironment() throws IOException;
+
+    /**
+     * Starts the NiFi instance and waits until the startup is complete
+     */
+    void start();
+
+    /**
+     * Shuts down the NiFi instance
+     */
+    void stop();
+
+    /**
+     * @return <code>true</code> if this node is part of a cluster (regardless of whether or not its current state is connected), <code>false</code> if the instance is a standalone instance
+     */
+    boolean isClustered();
+
+    /**
+     * @return the number of nodes in the cluster if this instance is part of a cluster, <code>1</code> otherwise
+     */
+    int getNumberOfNodes();
+
+    /**
+     * Returns the number of nodes in the cluster, optionally excluding instances that are not set to auto-start.
+     *
+     * @param includeOnlyAutoStartInstances whether or not to include instances that should not auto-start
+     *
+     * @return the number of nodes in the cluster
+     */
+    int getNumberOfNodes(boolean includeOnlyAutoStartInstances);
+
+    /**
+     * Returns the NiFiInstance for a specific node
+     *
+     * @param nodeIndex the 1-based index of the node
+     *
+     * @return the NiFi instance for the specified node
+     *
+     * @throws UnsupportedOperationException if the NiFi instance is not clustered
+     */
+    NiFiInstance getNodeInstance(int nodeIndex);
+
+    /**
+     * Returns the NiFiProperties for the node
+     *
+     * @return the nifi properties for the node
+     */
+    Properties getProperties() throws IOException;
+
+    /**
+     * @return the root directory for the instance
+     */
+    File getInstanceDirectory();
+
+    /**
+     * @return Whether or not this instance should automatically start when the test is run.
+     */
+    boolean isAutoStart();
+
+    /**
+     * Change the value of one of the properties in nifi.properties. If the node is already running, this change will not take effect until the instance is stopped and started again.
+     *
+     * @param propertyName the name of the property
+     * @param propertyValue the value of the property
+     */
+    void setProperty(String propertyName, String propertyValue) throws IOException;
+
+    /**
+     * Change the values of the given properties in nifi.properties. Any property that is not present in the given map will remain unchanged. If the node is already running, this change will not take
+     * effect until the instance is stopped and started again.
+     *
+     * @param properties the properties to change
+     */
+    void setProperties(Map<String, String> properties) throws IOException;
+}
diff --git a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/ControllerServicesClient.java b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/NiFiInstanceFactory.java
similarity index 59%
copy from nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/ControllerServicesClient.java
copy to nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/NiFiInstanceFactory.java
index af55421..61cd8d1 100644
--- a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/ControllerServicesClient.java
+++ b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/NiFiInstanceFactory.java
@@ -14,20 +14,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.nifi.toolkit.cli.impl.client.nifi;
-
-import org.apache.nifi.web.api.entity.ControllerServiceEntity;
-import org.apache.nifi.web.api.entity.ControllerServiceRunStatusEntity;
+package org.apache.nifi.tests.system;
 
 import java.io.IOException;
 
-/**
- * Client for interacting with NiFi's Controller Services Resource.
- */
-public interface ControllerServicesClient {
-
-    ControllerServiceEntity getControllerService(String id) throws NiFiClientException, IOException;
-
-    ControllerServiceEntity activateControllerService(String id, ControllerServiceRunStatusEntity runStatusEntity) throws NiFiClientException, IOException;
-
+public interface NiFiInstanceFactory {
+    NiFiInstance createInstance() throws IOException;
 }
diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/NiFiSystemIT.java b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/NiFiSystemIT.java
new file mode 100644
index 0000000..82a739c
--- /dev/null
+++ b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/NiFiSystemIT.java
@@ -0,0 +1,238 @@
+/*
+ * 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.
+ */
+package org.apache.nifi.tests.system;
+
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClient;
+import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientConfig;
+import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientException;
+import org.apache.nifi.toolkit.cli.impl.client.nifi.impl.JerseyNiFiClient;
+import org.apache.nifi.web.api.entity.ClusteSummaryEntity;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.rules.TestName;
+import org.junit.rules.Timeout;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.BooleanSupplier;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+public abstract class NiFiSystemIT {
+    public static final int CLIENT_API_PORT = 5671;
+    public static final String NIFI_GROUP_ID = "org.apache.nifi";
+    public static final String TEST_EXTENSIONS_ARTIFACT_ID = "nifi-system-test-extensions-nar";
+    public static final String TEST_PROCESSORS_PACKAGE = "org.apache.nifi.processors.tests.system";
+    public static final String TEST_CS_PACKAGE = "org.apache.nifi.cs.tests.system";
+
+    private static final Pattern FRAMEWORK_NAR_PATTERN = Pattern.compile("nifi-framework-nar-(.*?)\\.nar");
+    private static final File LIB_DIR = new File("target/nifi-lib-assembly/lib");
+    private static volatile String nifiFrameworkVersion = null;
+
+    protected static final Relationship REL_SUCCESS = new Relationship.Builder()
+        .name("success")
+        .description("Convenience Relationship for use in tests")
+        .build();
+
+    @Rule
+    public TestName name = new TestName();
+    @Rule
+    public Timeout defaultTimeout = new Timeout(2, TimeUnit.MINUTES);
+
+    private NiFiClient nifiClient;
+    private NiFiClientUtil clientUtil;
+    private static final AtomicReference<NiFiInstance> nifiRef = new AtomicReference<>();
+
+    @Before
+    public void setup() throws IOException {
+        Thread.currentThread().setContextClassLoader(ClassLoader.getSystemClassLoader());
+        setupClient();
+
+        if (nifiRef.get() == null) {
+            final NiFiInstance instance = getInstanceFactory().createInstance();
+            nifiRef.set(instance);
+            instance.createEnvironment();
+            instance.start();
+
+            Thread.currentThread().setContextClassLoader(ClassLoader.getSystemClassLoader());
+
+            if (instance.isClustered()) {
+                waitForAllNodesConnected();
+            }
+        }
+    }
+
+    @AfterClass
+    public static void cleanup() {
+        final NiFiInstance nifi = nifiRef.get();
+        nifiRef.set(null);
+        if (nifi != null) {
+            nifi.stop();
+        }
+    }
+
+    @After
+    public void teardown() throws IOException, NiFiClientException {
+        try {
+            if (isDestroyFlowAfterEachTest()) {
+                destroyFlow();
+            }
+        } finally {
+            if (nifiClient != null) {
+                nifiClient.close();
+            }
+        }
+    }
+
+    protected void destroyFlow() throws NiFiClientException, IOException {
+        getClientUtil().stopProcessGroupComponents("root");
+        getClientUtil().disableControllerServices("root");
+        getClientUtil().stopTransmitting("root");
+        getClientUtil().deleteAll("root");
+    }
+
+    protected void waitForAllNodesConnected() {
+        waitForAllNodesConnected(getNumberOfNodes(true));
+    }
+
+    protected void waitForAllNodesConnected(final int expectedNumberOfNodes) {
+        waitForAllNodesConnected(expectedNumberOfNodes, 100L);
+    }
+
+    protected void waitForAllNodesConnected(final int expectedNumberOfNodes, final long sleepMillis) {
+        final NiFiClient client = getNifiClient();
+
+        final long maxTime = System.currentTimeMillis() + TimeUnit.SECONDS.toMillis(60);
+        while (true) {
+            try {
+                final ClusteSummaryEntity clusterSummary = client.getFlowClient().getClusterSummary();
+                final int connectedNodeCount = clusterSummary.getClusterSummary().getConnectedNodeCount();
+                if (connectedNodeCount == expectedNumberOfNodes) {
+                    return;
+                }
+
+                if (System.currentTimeMillis() > maxTime) {
+                    throw new RuntimeException("Waited up to 60 seconds for both nodes to connect but only " + connectedNodeCount + " nodes connected");
+                }
+            } catch (final Exception e) {
+                e.printStackTrace();
+            }
+
+            try {
+                Thread.sleep(sleepMillis);
+            } catch (final InterruptedException ie) {
+                Thread.currentThread().interrupt();
+                return;
+            }
+        }
+    }
+
+    protected void setupClient() {
+        nifiClient = createClient();
+        clientUtil = new NiFiClientUtil(nifiClient, getNiFiVersion());
+    }
+
+    protected NiFiClientUtil getClientUtil() {
+        return clientUtil;
+    }
+
+    protected NiFiClient createClient() {
+        final NiFiClientConfig clientConfig = new NiFiClientConfig.Builder()
+            .baseUrl("http://localhost:" + getClientApiPort())
+            .connectTimeout(30000)
+            .readTimeout(30000)
+            .build();
+
+        return new JerseyNiFiClient.Builder()
+            .config(clientConfig)
+            .build();
+    }
+
+    protected int getClientApiPort() {
+        return CLIENT_API_PORT;
+    }
+
+
+    protected String getTestName() {
+        return name.getMethodName();
+    }
+
+    protected NiFiClient getNifiClient() {
+        Thread.currentThread().setContextClassLoader(ClassLoader.getSystemClassLoader());
+        return nifiClient;
+    }
+
+    protected String getNiFiVersion() {
+        final String knownVersion = nifiFrameworkVersion;
+        if (knownVersion != null) {
+            return knownVersion;
+        }
+
+        final File[] files = LIB_DIR.listFiles();
+        for (final File file : files) {
+            final String filename = file.getName();
+            final Matcher matcher = FRAMEWORK_NAR_PATTERN.matcher(filename);
+            if (matcher.matches()) {
+                final String version = matcher.group(1);
+                nifiFrameworkVersion = version;
+                return version;
+            }
+        }
+
+        throw new IllegalStateException("Could not determine version of NiFi");
+    }
+
+    protected NiFiInstance getNiFiInstance() {
+        return nifiRef.get();
+    }
+
+    protected int getNumberOfNodes() {
+        return getNumberOfNodes(true);
+    }
+
+    protected int getNumberOfNodes(final boolean includeOnlyAutoStartInstances) {
+        final NiFiInstance instance = nifiRef.get();
+        if (instance == null) {
+            return 1;
+        }
+
+        return instance.getNumberOfNodes(includeOnlyAutoStartInstances);
+    }
+
+    protected NiFiInstanceFactory getInstanceFactory() {
+        return new SpawnedStandaloneNiFiInstanceFactory(
+            new InstanceConfiguration.Builder()
+                .bootstrapConfig("src/test/resources/conf/default/bootstrap.conf")
+                .instanceDirectory("target/standalone-instance")
+                .build());
+    }
+
+    protected boolean isDestroyFlowAfterEachTest() {
+        return true;
+    }
+
+    protected void waitFor(final BooleanSupplier condition) throws InterruptedException {
+        while (!condition.getAsBoolean()) {
+            Thread.sleep(10L);
+        }
+    }
+}
diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/SpawnedClusterNiFiInstanceFactory.java b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/SpawnedClusterNiFiInstanceFactory.java
new file mode 100644
index 0000000..bb4939b
--- /dev/null
+++ b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/SpawnedClusterNiFiInstanceFactory.java
@@ -0,0 +1,66 @@
+/*
+ * 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.
+ */
+package org.apache.nifi.tests.system;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+public class SpawnedClusterNiFiInstanceFactory implements NiFiInstanceFactory {
+    private final List<InstanceConfiguration> instanceConfigs = new ArrayList<>();
+
+    public SpawnedClusterNiFiInstanceFactory(final String... bootstrapFilenames) {
+        int i = 0;
+        for (final String bootstrapFilename : bootstrapFilenames) {
+            final InstanceConfiguration config = new InstanceConfiguration.Builder()
+                .bootstrapConfig(bootstrapFilename)
+                .instanceDirectory("target/node" + (++i))
+                .build();
+
+            instanceConfigs.add(config);
+        }
+    }
+
+    public SpawnedClusterNiFiInstanceFactory(final String bootstrapFilename, final int nodeIndex) {
+        final InstanceConfiguration config = new InstanceConfiguration.Builder()
+            .bootstrapConfig(bootstrapFilename)
+            .instanceDirectory("target/node" + nodeIndex)
+            .build();
+
+        instanceConfigs.add(config);
+    }
+
+    public SpawnedClusterNiFiInstanceFactory(final InstanceConfiguration... instanceConfigurations) {
+        instanceConfigs.addAll(Arrays.asList(instanceConfigurations));
+    }
+
+    @Override
+    public NiFiInstance createInstance() {
+        final List<NiFiInstance> instances = new ArrayList<>();
+
+        for (final InstanceConfiguration configuration : instanceConfigs) {
+            final NiFiInstance clusteredInstance = new SpawnedStandaloneNiFiInstanceFactory(configuration).createInstance();
+            instances.add(clusteredInstance);
+        }
+
+        if (instances.size() == 1) {
+            return instances.get(0);
+        }
+
+        return new AggregateNiFiInstance(instances);
+    }
+}
diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/SpawnedStandaloneNiFiInstanceFactory.java b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/SpawnedStandaloneNiFiInstanceFactory.java
new file mode 100644
index 0000000..bf5c40e
--- /dev/null
+++ b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/SpawnedStandaloneNiFiInstanceFactory.java
@@ -0,0 +1,293 @@
+/*
+ * 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.
+ */
+package org.apache.nifi.tests.system;
+
+import org.apache.nifi.bootstrap.RunNiFi;
+import org.apache.nifi.registry.security.util.KeystoreType;
+import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClient;
+import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientConfig;
+import org.apache.nifi.toolkit.cli.impl.client.nifi.impl.JerseyNiFiClient;
+import org.apache.nifi.util.file.FileUtils;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Properties;
+
+import static org.junit.Assert.assertTrue;
+
+public class SpawnedStandaloneNiFiInstanceFactory implements NiFiInstanceFactory {
+    private final InstanceConfiguration instanceConfig;
+
+    public SpawnedStandaloneNiFiInstanceFactory(final InstanceConfiguration instanceConfig) {
+        this.instanceConfig = instanceConfig;
+    }
+
+    @Override
+    public NiFiInstance createInstance() {
+        return new RunNiFiInstance(instanceConfig);
+    }
+
+
+    private static class RunNiFiInstance implements NiFiInstance {
+        private final File instanceDirectory;
+        private final File configDir;
+        private final InstanceConfiguration instanceConfiguration;
+        private File bootstrapConfigFile;
+        private RunNiFi runNiFi;
+
+        public RunNiFiInstance(final InstanceConfiguration instanceConfiguration) {
+            this.instanceDirectory = instanceConfiguration.getInstanceDirectory();
+            this.bootstrapConfigFile = instanceConfiguration.getBootstrapConfigFile();
+            this.instanceConfiguration = instanceConfiguration;
+
+            final Properties bootstrapProperties = new Properties();
+            try (final InputStream fis = new FileInputStream(bootstrapConfigFile)) {
+                bootstrapProperties.load(fis);
+            } catch (final IOException e) {
+                throw new RuntimeException("Could not load boostrap config file " + bootstrapConfigFile, e);
+            }
+
+            final String confDirName = bootstrapProperties.getProperty("conf.dir");
+            final File tempConfDir = new File(confDirName);
+            if (tempConfDir.isAbsolute()) {
+                configDir = tempConfDir;
+            } else {
+                configDir = new File(instanceDirectory, confDirName);
+            }
+        }
+
+        @Override
+        public void start() {
+            if (runNiFi != null) {
+                throw new IllegalStateException("NiFi has already been started");
+            }
+
+            System.out.println("Starting instance " + instanceDirectory.getName());
+
+            try {
+                this.runNiFi = new RunNiFi(bootstrapConfigFile);
+            } catch (IOException e) {
+                throw new RuntimeException("Failed to start NiFi", e);
+            }
+
+            try {
+                runNiFi.start(false);
+                waitForStartup();
+            } catch (IOException e) {
+                throw new RuntimeException("Failed to start NiFi", e);
+            }
+        }
+
+        public void createEnvironment() throws IOException {
+            System.out.println("Creating environment for instance " + instanceDirectory.getName());
+
+            cleanup();
+
+            final File destinationConf = new File(instanceDirectory, "conf");
+            copyContents(bootstrapConfigFile.getParentFile(), destinationConf);
+            bootstrapConfigFile = new File(destinationConf, bootstrapConfigFile.getName());
+
+            final File destinationLib = new File(instanceDirectory, "lib");
+            copyContents(new File("target/nifi-lib-assembly/lib"), destinationLib);
+
+            final File destinationCertsDir = new File(instanceDirectory, "certs");
+            if (!destinationCertsDir.exists()) {
+                assertTrue(destinationCertsDir.mkdirs());
+            }
+
+            // Copy keystore
+            final File destinationKeystore = new File(destinationCertsDir, "keystore.jks");
+            Files.copy(Paths.get("src/test/resources/keystore.jks"), destinationKeystore.toPath());
+
+            // Copy truststore
+            final File destinationTruststore = new File(destinationCertsDir, "truststore.jks");
+            Files.copy(Paths.get("src/test/resources/truststore.jks"), destinationTruststore.toPath());
+        }
+
+        private void copyContents(final File dir, final File destinationDir) throws IOException {
+            if (!destinationDir.exists()) {
+                assertTrue(destinationDir.mkdirs());
+            }
+
+            final File[] sourceFiles = dir.listFiles();
+            for (final File sourceFile : sourceFiles) {
+                if (sourceFile.isDirectory()) {
+                    final File destinationFile = new File(destinationDir, sourceFile.getName());
+                    copyContents(sourceFile, destinationFile);
+                    continue;
+                }
+
+                final File destinationFile = new File(destinationDir, sourceFile.getName());
+                if (destinationFile.exists()) {
+                    assertTrue(destinationFile.delete());
+                }
+
+                Files.copy(sourceFile.toPath(), destinationFile.toPath());
+            }
+        }
+
+
+        private void waitForStartup() throws IOException {
+            final NiFiClient client = createClient();
+
+            while (true) {
+                try {
+                    client.getFlowClient().getRootGroupId();
+                    System.out.println("Completed startup of instance " + instanceDirectory.getName());
+                    return;
+                } catch (final Exception e) {
+                    try {
+                        Thread.sleep(100L);
+                    } catch (InterruptedException ex) {
+                    }
+
+                    continue;
+                }
+            }
+        }
+
+        @Override
+        public void stop() {
+            if (runNiFi == null) {
+                return;
+            }
+
+            System.out.println("Stopping instance " + instanceDirectory.getName());
+
+            try {
+                runNiFi.stop();
+                System.out.println("Completed shutdown of instance " + instanceDirectory.getName());
+            } catch (IOException e) {
+                throw new RuntimeException("Failed to stop NiFi", e);
+            } finally {
+                runNiFi = null;
+            }
+        }
+
+
+        private void cleanup() throws IOException {
+            if (instanceDirectory.exists()) {
+                FileUtils.deleteFile(instanceDirectory, true);
+            }
+        }
+
+        @Override
+        public boolean isClustered() {
+            return false;
+        }
+
+        @Override
+        public int getNumberOfNodes() {
+            return 1;
+        }
+
+        @Override
+        public int getNumberOfNodes(final boolean includeOnlyAutoStartInstances) {
+            return isAutoStart() ? 1 : 0;
+        }
+
+        @Override
+        public NiFiInstance getNodeInstance(final int nodeIndex) {
+            return null;
+        }
+
+        @Override
+        public Properties getProperties() throws IOException {
+            final File nifiPropsFile = new File(configDir, "nifi.properties");
+            final Properties nifiProps = new Properties();
+
+            try (final InputStream fis = new FileInputStream(nifiPropsFile)) {
+                nifiProps.load(fis);
+            }
+
+            return nifiProps;
+        }
+
+        @Override
+        public File getInstanceDirectory() {
+            return instanceDirectory;
+        }
+
+        @Override
+        public boolean isAutoStart() {
+            return instanceConfiguration.isAutoStart();
+        }
+
+        @Override
+        public void setProperty(final String propertyName, final String propertyValue) throws IOException {
+            setProperties(Collections.singletonMap(propertyName, propertyValue));
+        }
+
+        @Override
+        public void setProperties(final Map<String, String> properties) throws IOException {
+            final Properties currentProperties = getProperties();
+            currentProperties.putAll(properties);
+
+            final File propertiesFile = new File(configDir, "nifi.properties");
+            try (final OutputStream fos = new FileOutputStream(propertiesFile)) {
+                currentProperties.store(fos, "");
+            }
+        }
+
+        public NiFiClient createClient() throws IOException {
+            final Properties nifiProperties = getProperties();
+            final String httpPort = nifiProperties.getProperty("nifi.web.http.port");
+            final String httpsPort = nifiProperties.getProperty("nifi.web.https.port");
+            final String webPort = (httpsPort == null || httpsPort.trim().isEmpty()) ? httpPort : httpsPort;
+
+            final String keystoreType = nifiProperties.getProperty("nifi.security.keystoreType");
+            final String truststoreType = nifiProperties.getProperty("nifi.security.truststoreType");
+
+            final NiFiClientConfig clientConfig = new NiFiClientConfig.Builder()
+                .baseUrl("http://localhost:" + webPort)
+                .connectTimeout(30000)
+                .readTimeout(30000)
+                .keystoreFilename(getAbsolutePath(nifiProperties.getProperty("nifi.security.keystore")))
+                .keystorePassword(nifiProperties.getProperty("nifi.security.keystorePasswd"))
+                .keystoreType(keystoreType == null ? null : KeystoreType.valueOf(keystoreType))
+                .truststoreFilename(getAbsolutePath(nifiProperties.getProperty("nifi.security.truststore")))
+                .truststorePassword(nifiProperties.getProperty("nifi.security.truststorePasswd"))
+                .truststoreType(truststoreType == null ? null : KeystoreType.valueOf(truststoreType))
+                .build();
+
+            return new JerseyNiFiClient.Builder()
+                .config(clientConfig)
+                .build();
+        }
+
+        private String getAbsolutePath(final String filename) {
+            if (filename == null) {
+                return null;
+            }
+
+            final File file = new File(filename);
+            if (file.isAbsolute()) {
+                return file.getAbsolutePath();
+            }
+
+            return new File(instanceDirectory, file.getPath()).getAbsolutePath();
+        }
+    }
+}
diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/clustering/JoinClusterAdjustStateIT.java b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/clustering/JoinClusterAdjustStateIT.java
new file mode 100644
index 0000000..5849a52
--- /dev/null
+++ b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/clustering/JoinClusterAdjustStateIT.java
@@ -0,0 +1,105 @@
+/*
+ * 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.
+ */
+package org.apache.nifi.tests.system.clustering;
+
+import org.apache.nifi.tests.system.InstanceConfiguration;
+import org.apache.nifi.tests.system.NiFiInstance;
+import org.apache.nifi.tests.system.NiFiInstanceFactory;
+import org.apache.nifi.tests.system.NiFiSystemIT;
+import org.apache.nifi.tests.system.SpawnedClusterNiFiInstanceFactory;
+import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientException;
+import org.apache.nifi.web.api.entity.ProcessorEntity;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.Collections;
+
+import static org.junit.Assert.assertTrue;
+
+public class JoinClusterAdjustStateIT extends NiFiSystemIT {
+    @Override
+    protected NiFiInstanceFactory getInstanceFactory() {
+        return new SpawnedClusterNiFiInstanceFactory(
+            new InstanceConfiguration.Builder()
+                .bootstrapConfig("src/test/resources/conf/clustered/node1/bootstrap.conf")
+                .instanceDirectory("target/node1")
+                .build(),
+            new InstanceConfiguration.Builder()
+                .bootstrapConfig("src/test/resources/conf/clustered/node2/bootstrap.conf")
+                .instanceDirectory("target/node2")
+                .autoStart(false)
+                .build()
+        );
+    }
+
+    @Test
+    public void testProcessorsStartWhenAble() throws NiFiClientException, IOException, InterruptedException {
+        final ProcessorEntity countProcessor = getClientUtil().createProcessor(TEST_PROCESSORS_PACKAGE + ".CountEvents", NIFI_GROUP_ID, TEST_EXTENSIONS_ARTIFACT_ID, getNiFiVersion());
+        final ProcessorEntity fileProcessor = getClientUtil().createProcessor(TEST_PROCESSORS_PACKAGE + ".ValidateFileExists", NIFI_GROUP_ID, TEST_EXTENSIONS_ARTIFACT_ID, getNiFiVersion());
+
+        // Set scheduling Period to 1 hour so that we know when the Processors actually get triggered to run.
+        getClientUtil().updateProcessorSchedulingPeriod(countProcessor, "1 hour");
+        getClientUtil().updateProcessorSchedulingPeriod(fileProcessor, "1 hour");
+
+        final File firstNodeInstanceDir = getNiFiInstance().getNodeInstance(1).getInstanceDirectory();
+        final File firstNodeMonitoredDir = new File(firstNodeInstanceDir, "monitored");
+        assertTrue(firstNodeMonitoredDir.mkdirs());
+
+        getClientUtil().updateProcessorProperties(fileProcessor, Collections.singletonMap("Filename", "./monitored"));
+        getClientUtil().setAutoTerminatedRelationships(countProcessor, Collections.singleton("success"));
+        getClientUtil().setAutoTerminatedRelationships(fileProcessor, Collections.singleton("success"));
+
+        getClientUtil().waitForValidProcessor(countProcessor.getId());
+        getClientUtil().waitForValidProcessor(fileProcessor.getId());
+
+        // Start the Processor that requires a file named "monitored" to exist. When we join Node 2 to the cluster, this directory will not exist.
+        // We want to ensure that the Processor does in fact start on its own when the directory is created.
+        getNifiClient().getProcessorClient().startProcessor(fileProcessor.getId(), fileProcessor.getRevision().getClientId(), 1);
+        getClientUtil().waitForRunningProcessor(fileProcessor.getId());
+
+        // Create a new NiFi instance
+        final NiFiInstance node2Instance = getNiFiInstance().getNodeInstance(2);
+
+        // Copy the flow from Node 1 to Node 2.
+        final File node1Flow = new File(firstNodeInstanceDir, "conf/flow.xml.gz");
+        final File node2Flow = new File(node2Instance.getInstanceDirectory(), "conf/flow.xml.gz");
+        Thread.sleep(2000L); // Wait a bit before copying it, since the flow is written out in the background, and we want to ensure that the flow is up-to-date.
+        Files.copy(node1Flow.toPath(), node2Flow.toPath());
+
+        // Start the Count Processor on Node 1. When Node 2 joins the cluster, we know that its flow will indicate that the Processor is stopped.
+        // But because the cluster indicates that the Processor is running, the second node should inherit this value and immediately start the Processor also.
+        getNifiClient().getProcessorClient().startProcessor(countProcessor.getId(), countProcessor.getRevision().getClientId(), 1);
+
+        node2Instance.start();
+
+        // Wait for the second node to join the cluster.
+        waitForAllNodesConnected(2);
+
+        // Wait for the Count Processor to be running
+        getClientUtil().waitForRunningProcessor(countProcessor.getId());
+        getClientUtil().waitForCounter(countProcessor.getId(), "Triggered", 2);
+
+        // Create the "monitored" directory for node 2.
+        final File secondNodeMonitoredDir = new File(node2Instance.getInstanceDirectory(), "monitored");
+        assertTrue(secondNodeMonitoredDir.mkdirs());
+
+        getClientUtil().waitForCounter(fileProcessor.getId(), "Triggered", 2);
+    }
+
+}
diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/clustering/NodeRestartWithNewNodeIdIT.java b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/clustering/NodeRestartWithNewNodeIdIT.java
new file mode 100644
index 0000000..a1a608b
--- /dev/null
+++ b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/clustering/NodeRestartWithNewNodeIdIT.java
@@ -0,0 +1,102 @@
+/*
+ * 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.
+ */
+package org.apache.nifi.tests.system.clustering;
+
+import org.apache.nifi.tests.system.NiFiInstance;
+import org.apache.nifi.tests.system.NiFiInstanceFactory;
+import org.apache.nifi.tests.system.NiFiSystemIT;
+import org.apache.nifi.tests.system.SpawnedClusterNiFiInstanceFactory;
+import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientException;
+import org.apache.nifi.web.api.dto.NodeDTO;
+import org.apache.nifi.web.api.entity.ClusteSummaryEntity;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Collection;
+
+import static junit.framework.TestCase.assertTrue;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+
+public class NodeRestartWithNewNodeIdIT extends NiFiSystemIT {
+
+    @Override
+    protected NiFiInstanceFactory getInstanceFactory() {
+        return new SpawnedClusterNiFiInstanceFactory(
+            "src/test/resources/conf/clustered/node1/bootstrap.conf",
+            "src/test/resources/conf/clustered/node2/bootstrap.conf");
+    }
+
+    @Test
+    public void testRestartNodeWithDifferentNodeId() throws IOException, NiFiClientException {
+        // Get a view of the cluster as it is currently.
+        final Collection<NodeDTO> originalNodeDTOs = getNifiClient().getControllerClient().getNodes().getCluster().getNodes();
+        final NodeDTO originalNode1 = originalNodeDTOs.stream().filter(dto -> dto.getApiPort() == 5671).findFirst().orElse(null);
+        final NodeDTO originalNode2 = originalNodeDTOs.stream().filter(dto -> dto.getApiPort() == 5672).findFirst().orElse(null);
+
+        // Stop the second instance, delete its state directory, and restart it.
+        final NiFiInstance secondNode = getNiFiInstance().getNodeInstance(2);
+        secondNode.stop();
+
+        final File destinationDir = new File(secondNode.getInstanceDirectory(), "state/local");
+        deleteChildren(destinationDir);
+        secondNode.start();
+
+        // Wait for the second node to reconnect.
+        waitForAllNodesConnected();
+
+        final ClusteSummaryEntity clusterSummary = getNifiClient().getFlowClient().getClusterSummary();
+        // 2/2 because we replaced the state directory with a different state that had a different identifier. This will result in
+        // 2 nodes each with a different UUID but the same hostname & port, but we should see that the cluster removes the old Node
+        // Identifier, so we will have only 2 nodes total.
+        assertEquals("2 / 2", clusterSummary.getClusterSummary().getConnectedNodes());
+
+        // Get the updated view of the cluster.
+        final Collection<NodeDTO> updatedNodeDTOs = getNifiClient().getControllerClient().getNodes().getCluster().getNodes();
+
+        // Node 1 should be the same, but Node 2 should have a new UUID. This is because on startup, the node will not have found any local state. As a result, it
+        // will create a new UUID for its Node Identifier. However, it should have the same hostname and port. As a result, the cluster will remove the old Node that has
+        // a different UUID but the same hostname and port.
+        final NodeDTO updatedNode1 = updatedNodeDTOs.stream().filter(dto -> dto.getApiPort() == 5671).findFirst().orElse(null);
+        final NodeDTO updatedNode2 = updatedNodeDTOs.stream().filter(dto -> dto.getApiPort() == 5672).findFirst().orElse(null);
+
+        assertEquals(originalNode1.getNodeId(), updatedNode1.getNodeId());
+        assertEquals(originalNode1.getAddress(), updatedNode1.getAddress());
+        assertEquals(originalNode1.getApiPort(), updatedNode1.getApiPort());
+
+        assertNotEquals(originalNode2.getNodeId(), updatedNode2.getNodeId()); // the UUID of Node 2 will be different, but the hostnames & ports should be the same.
+        assertEquals(originalNode2.getAddress(), updatedNode2.getAddress());
+        assertEquals(originalNode2.getApiPort(), updatedNode2.getApiPort());
+    }
+
+
+    private void deleteChildren(final File dir) {
+        if (!dir.exists()) {
+            return;
+        }
+
+        for (final File file : dir.listFiles()) {
+            if (file.isDirectory()) {
+                deleteChildren(file);
+            }
+
+            assertTrue(file.delete());
+        }
+    }
+
+}
diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/clustering/RestartWithDifferentPort.java b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/clustering/RestartWithDifferentPort.java
new file mode 100644
index 0000000..c0c5b5d
--- /dev/null
+++ b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/clustering/RestartWithDifferentPort.java
@@ -0,0 +1,79 @@
+/*
+ * 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.
+ */
+package org.apache.nifi.tests.system.clustering;
+
+import org.apache.nifi.tests.system.NiFiInstance;
+import org.apache.nifi.tests.system.NiFiInstanceFactory;
+import org.apache.nifi.tests.system.NiFiSystemIT;
+import org.apache.nifi.tests.system.SpawnedClusterNiFiInstanceFactory;
+import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientException;
+import org.apache.nifi.web.api.dto.NodeDTO;
+import org.apache.nifi.web.api.entity.ClusteSummaryEntity;
+import org.apache.nifi.web.api.entity.ClusterEntity;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class RestartWithDifferentPort extends NiFiSystemIT {
+    @Override
+    protected NiFiInstanceFactory getInstanceFactory() {
+        return new SpawnedClusterNiFiInstanceFactory(
+            "src/test/resources/conf/clustered/node1/bootstrap.conf",
+            "src/test/resources/conf/clustered/node2/bootstrap.conf");
+    }
+
+    @Test
+    public void testRestartWithDifferentPortKeepsNodeIdUnchanged() throws IOException, NiFiClientException {
+        // Get the set of Node UUID's
+        ClusterEntity clusterEntity = getNifiClient().getControllerClient().getNodes();
+        Collection<NodeDTO> nodeDtos = clusterEntity.getCluster().getNodes();
+        final Set<String> nodeUuids = nodeDtos.stream().map(NodeDTO::getNodeId).collect(Collectors.toSet());
+
+        // Stop the second instance and change its web api port
+        final NiFiInstance secondNode = getNiFiInstance().getNodeInstance(2);
+        secondNode.stop();
+
+        // Change the value of the nifi.web.http.port property from 5672 to 5673
+        secondNode.setProperty("nifi.web.http.port", "5673");
+
+        // Restart the second node
+        secondNode.start();
+        waitForAllNodesConnected(getNumberOfNodes(true), 2000L);
+
+        // Wait for the second node to reconnect.
+        final ClusteSummaryEntity clusterSummary = getNifiClient().getFlowClient().getClusterSummary();
+        assertEquals("2 / 2", clusterSummary.getClusterSummary().getConnectedNodes());
+
+        // Ensure that the Node UUID's are the same and that we now have 2 nodes: localhost:5671 and localhost:5673, but NOT localhost:5672
+        clusterEntity = getNifiClient().getControllerClient().getNodes();
+        nodeDtos = clusterEntity.getCluster().getNodes();
+
+        final Set<String> updatedNodeUuids = nodeDtos.stream().map(NodeDTO::getNodeId).collect(Collectors.toSet());
+        assertEquals(nodeUuids, updatedNodeUuids);
+
+        final Set<String> nodeAddresses = nodeDtos.stream().map(dto -> dto.getAddress() + ":" + dto.getApiPort()).collect(Collectors.toSet());
+        assertEquals(2, nodeAddresses.size());
+        assertTrue(nodeAddresses.contains("localhost:5671"));
+        assertTrue(nodeAddresses.contains("localhost:5673"));
+    }
+}
diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/clustering/SimpleNodeRestartIT.java b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/clustering/SimpleNodeRestartIT.java
new file mode 100644
index 0000000..cb6c482
--- /dev/null
+++ b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/clustering/SimpleNodeRestartIT.java
@@ -0,0 +1,52 @@
+/*
+ * 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.
+ */
+package org.apache.nifi.tests.system.clustering;
+
+import org.apache.nifi.tests.system.SpawnedClusterNiFiInstanceFactory;
+import org.apache.nifi.tests.system.NiFiInstance;
+import org.apache.nifi.tests.system.NiFiInstanceFactory;
+import org.apache.nifi.tests.system.NiFiSystemIT;
+import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientException;
+import org.apache.nifi.web.api.entity.ClusteSummaryEntity;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.junit.Assert.assertEquals;
+
+public class SimpleNodeRestartIT extends NiFiSystemIT {
+
+    @Override
+    protected NiFiInstanceFactory getInstanceFactory() {
+        return new SpawnedClusterNiFiInstanceFactory(
+            "src/test/resources/conf/clustered/node1/bootstrap.conf",
+            "src/test/resources/conf/clustered/node2/bootstrap.conf");
+    }
+
+
+    @Test
+    public void testRestartNode() throws NiFiClientException, IOException {
+        final NiFiInstance secondNode = getNiFiInstance().getNodeInstance(2);
+        secondNode.stop();
+
+        secondNode.start();
+        waitForAllNodesConnected();
+
+        final ClusteSummaryEntity clusterSummary = getNifiClient().getFlowClient().getClusterSummary();
+        assertEquals("2 / 2", clusterSummary.getClusterSummary().getConnectedNodes());
+    }
+}
diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/controllerservice/DependentControllerServiceIT.java b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/controllerservice/DependentControllerServiceIT.java
new file mode 100644
index 0000000..90950f7
--- /dev/null
+++ b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/controllerservice/DependentControllerServiceIT.java
@@ -0,0 +1,80 @@
+/*
+ * 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.
+ */
+package org.apache.nifi.tests.system.controllerservice;
+
+import org.apache.nifi.tests.system.NiFiSystemIT;
+import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientException;
+import org.apache.nifi.web.api.dto.RevisionDTO;
+import org.apache.nifi.web.api.entity.ControllerServiceEntity;
+import org.apache.nifi.web.api.entity.ControllerServiceReferencingComponentEntity;
+import org.apache.nifi.web.api.entity.ControllerServiceReferencingComponentsEntity;
+import org.apache.nifi.web.api.entity.ControllerServiceRunStatusEntity;
+import org.apache.nifi.web.api.entity.UpdateControllerServiceReferenceRequestEntity;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+public class DependentControllerServiceIT extends NiFiSystemIT {
+
+    @Test
+    public void testEnableDisableServiceWithReferences() throws NiFiClientException, IOException {
+        final ControllerServiceEntity sleepOnValidation = getClientUtil().createControllerService("StandardSleepService");
+        sleepOnValidation.getComponent().setProperties(Collections.singletonMap("Validate Sleep Time", "10 secs"));
+        getNifiClient().getControllerServicesClient().updateControllerService(sleepOnValidation);
+
+        final ControllerServiceEntity referencingService = getClientUtil().createControllerService("StandardSleepService");
+        referencingService.getComponent().setProperties(Collections.singletonMap("Dependent Service", sleepOnValidation.getId()));
+        getNifiClient().getControllerServicesClient().updateControllerService(referencingService);
+
+        // Enable Sleep on Validate
+        final ControllerServiceRunStatusEntity runStatusEntity = new ControllerServiceRunStatusEntity();
+        runStatusEntity.setState("ENABLED");
+        runStatusEntity.setRevision(sleepOnValidation.getRevision());
+        getNifiClient().getControllerServicesClient().activateControllerService(sleepOnValidation.getId(), runStatusEntity);
+
+        // Enable Referencing Services
+        final ControllerServiceReferencingComponentsEntity referencingComponentsEntity = getNifiClient().getControllerServicesClient().getControllerServiceReferences(sleepOnValidation.getId());
+
+        final Map<String, RevisionDTO> referenceRevisions = new HashMap<>();
+        for (final ControllerServiceReferencingComponentEntity componentEntity : referencingComponentsEntity.getControllerServiceReferencingComponents()) {
+            referenceRevisions.put(componentEntity.getId(), componentEntity.getRevision());
+        }
+
+        final UpdateControllerServiceReferenceRequestEntity updateReferencesEntity = new UpdateControllerServiceReferenceRequestEntity();
+        updateReferencesEntity.setId(sleepOnValidation.getId());
+        updateReferencesEntity.setReferencingComponentRevisions(referenceRevisions);
+        updateReferencesEntity.setState("ENABLED");
+        getNifiClient().getControllerServicesClient().updateControllerServiceReferences(updateReferencesEntity);
+
+        getClientUtil().waitForControllerSerivcesEnabled("root");
+
+        // Disable the referencing services.
+        updateReferencesEntity.setState("DISABLED");
+        getNifiClient().getControllerServicesClient().updateControllerServiceReferences(updateReferencesEntity);
+        getClientUtil().waitForControllerSerivcesDisabled("root", referencingService.getId());
+
+        // Disable the Sleep On Validation Service.
+        runStatusEntity.setState("DISABLED");
+        getNifiClient().getControllerServicesClient().activateControllerService(sleepOnValidation.getId(), runStatusEntity);
+
+        // Wait for all services to become disabled.
+        getClientUtil().waitForControllerSerivcesDisabled("root");
+    }
+}
diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/loadbalance/LoadBalanceIT.java b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/loadbalance/LoadBalanceIT.java
new file mode 100644
index 0000000..2a8b698
--- /dev/null
+++ b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/loadbalance/LoadBalanceIT.java
@@ -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.
+ */
+package org.apache.nifi.tests.system.loadbalance;
+
+import org.apache.nifi.controller.queue.LoadBalanceCompression;
+import org.apache.nifi.controller.queue.LoadBalanceStrategy;
+import org.apache.nifi.scheduling.ExecutionNode;
+import org.apache.nifi.tests.system.NiFiInstanceFactory;
+import org.apache.nifi.tests.system.NiFiSystemIT;
+import org.apache.nifi.tests.system.SpawnedClusterNiFiInstanceFactory;
+import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientException;
+import org.apache.nifi.web.api.dto.ConnectionDTO;
+import org.apache.nifi.web.api.dto.NodeDTO;
+import org.apache.nifi.web.api.dto.status.ConnectionStatusDTO;
+import org.apache.nifi.web.api.dto.status.ConnectionStatusSnapshotDTO;
+import org.apache.nifi.web.api.dto.status.NodeConnectionStatusSnapshotDTO;
+import org.apache.nifi.web.api.entity.ClusterEntity;
+import org.apache.nifi.web.api.entity.ConnectionEntity;
+import org.apache.nifi.web.api.entity.ConnectionStatusEntity;
+import org.apache.nifi.web.api.entity.ProcessorEntity;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.LongSummaryStatistics;
+import java.util.Map;
+
+import static junit.framework.TestCase.assertTrue;
+import static org.junit.Assert.assertEquals;
+
+public class LoadBalanceIT extends NiFiSystemIT {
+    @Override
+    protected NiFiInstanceFactory getInstanceFactory() {
+        return new SpawnedClusterNiFiInstanceFactory(
+            "src/test/resources/conf/clustered/node1/bootstrap.conf",
+            "src/test/resources/conf/clustered/node2/bootstrap.conf");
+    }
+
+    @Test
+    public void testRoundRobinStrategyNoCompression() throws NiFiClientException, IOException, InterruptedException {
+        testRoundRobinStrategy(LoadBalanceCompression.DO_NOT_COMPRESS);
+    }
+
+    @Test
+    public void testRoundRobinStrategyCompressAttributesOnly() throws NiFiClientException, IOException, InterruptedException {
+        testRoundRobinStrategy(LoadBalanceCompression.COMPRESS_ATTRIBUTES_ONLY);
+    }
+
+    @Test
+    public void testRoundRobinStrategyCompressAll() throws NiFiClientException, IOException, InterruptedException {
+        testRoundRobinStrategy(LoadBalanceCompression.COMPRESS_ATTRIBUTES_AND_CONTENT);
+    }
+
+    private void testRoundRobinStrategy(final LoadBalanceCompression compression) throws NiFiClientException, IOException, InterruptedException {
+        final ProcessorEntity generate = getClientUtil().createProcessor("GenerateFlowFile");
+        final ProcessorEntity count = getClientUtil().createProcessor("CountEvents");
+
+        final ConnectionEntity connection = getClientUtil().createConnection(generate, count, "success");
+        getClientUtil().setAutoTerminatedRelationships(count, "success");
+
+        // Configure Processor to generate 20 FlowFiles, each 1 MB and run on Primary Node.
+        final Map<String, String> generateProperties = new HashMap<>();
+        generateProperties.put("File Size", "1 MB");
+        generateProperties.put("Batch Size", "20");
+        getClientUtil().updateProcessorProperties(generate, generateProperties);
+        getClientUtil().updateProcessorExecutionNode(generate, ExecutionNode.PRIMARY);
+
+        // Round Robin between nodes. This should result in 10 FlowFiles on each node.
+        getClientUtil().updateConnectionLoadBalancing(connection, LoadBalanceStrategy.ROUND_ROBIN, compression, null);
+
+        // Generate the data.
+        getNifiClient().getProcessorClient().startProcessor(generate);
+
+        // Wait until all 20 FlowFiles are queued up.
+        waitFor(() -> {
+            final ConnectionStatusEntity statusEntity = getConnectionStatus(connection.getId());
+            return statusEntity.getConnectionStatus().getAggregateSnapshot().getFlowFilesQueued() == 20;
+        });
+
+        // Wait until load balancing is complete
+        waitFor(() -> isConnectionDoneLoadBalancing(connection.getId()));
+
+        // Ensure that the FlowFiles are evenly distributed between the nodes.
+        final ConnectionStatusEntity statusEntity = getConnectionStatus(connection.getId());
+        assertTrue(isEvenlyDistributed(statusEntity));
+
+        assertEquals(20, getQueueSize(connection.getId()));
+        assertEquals(20 * 1024 * 1024, getQueueBytes(connection.getId()));
+    }
+
+
+    @Test
+    public void testSingleNodeStrategy() throws NiFiClientException, IOException, InterruptedException {
+        final ProcessorEntity generate = getClientUtil().createProcessor("GenerateFlowFile");
+        final ProcessorEntity count = getClientUtil().createProcessor("CountEvents");
+
+        final ConnectionEntity connection = getClientUtil().createConnection(generate, count, "success");
+        getClientUtil().setAutoTerminatedRelationships(count, "success");
+
+        // Configure Processor to generate 10 FlowFiles, each 1 MB, on each node, for a total of 20 FlowFiles.
+        final Map<String, String> generateProperties = new HashMap<>();
+        generateProperties.put("File Size", "1 MB");
+        generateProperties.put("Batch Size", "10");
+        getClientUtil().updateProcessorProperties(generate, generateProperties);
+
+        // Round Robin between nodes. This should result in 10 FlowFiles on each node.
+        getClientUtil().updateConnectionLoadBalancing(connection, LoadBalanceStrategy.SINGLE_NODE, LoadBalanceCompression.DO_NOT_COMPRESS, null);
+
+        // Generate the data.
+        getNifiClient().getProcessorClient().startProcessor(generate);
+
+        // Wait until all 20 FlowFiles are queued up.
+        waitFor(() -> {
+            final ConnectionStatusEntity statusEntity = getConnectionStatus(connection.getId());
+            return statusEntity.getConnectionStatus().getAggregateSnapshot().getFlowFilesQueued() == 20;
+        });
+
+        // Wait until load balancing is complete
+        waitFor(() -> isConnectionDoneLoadBalancing(connection.getId()));
+
+        // Ensure that all FlowFiles are on the same node.
+        final ConnectionStatusEntity statusEntity = getConnectionStatus(connection.getId());
+        final ConnectionStatusDTO connectionStatusDto = statusEntity.getConnectionStatus();
+        final int numNodes = connectionStatusDto.getNodeSnapshots().size();
+
+        int emptyNodes = 0;
+
+        for (final NodeConnectionStatusSnapshotDTO nodeStatusDto : connectionStatusDto.getNodeSnapshots()) {
+            final ConnectionStatusSnapshotDTO snapshotDto = nodeStatusDto.getStatusSnapshot();
+            final int flowFilesQueued = snapshotDto.getFlowFilesQueued();
+
+            // Number of flowfiles should either be 0 or should be equal to the total number of FlowFiles in the queue.
+            if (flowFilesQueued == 0) {
+                emptyNodes++;
+            } else {
+                assertEquals(statusEntity.getConnectionStatus().getAggregateSnapshot().getFlowFilesQueued().intValue(), flowFilesQueued);
+            }
+        }
+
+        // Number of empty nodes should be one less than total number of nodes.
+        assertEquals(numNodes - 1, emptyNodes);
+    }
+
+    @Test
+    public void testOffload() throws NiFiClientException, IOException, InterruptedException {
+        final ProcessorEntity generate = getClientUtil().createProcessor("GenerateFlowFile");
+        final ProcessorEntity count = getClientUtil().createProcessor("CountEvents");
+
+        final ConnectionEntity connection = getClientUtil().createConnection(generate, count, "success");
+        getClientUtil().setAutoTerminatedRelationships(count, "success");
+
+        // Configure Processor to generate 10 FlowFiles, each 1 MB, on each node (for a total of 20)
+        final Map<String, String> generateProperties = new HashMap<>();
+        generateProperties.put("File Size", "1 MB");
+        generateProperties.put("Batch Size", "10");
+        getClientUtil().updateProcessorProperties(generate, generateProperties);
+
+        // Generate the data.
+        getNifiClient().getProcessorClient().startProcessor(generate);
+
+        // Wait until all 20 FlowFiles are queued up.
+        waitFor(() -> {
+            final ConnectionStatusEntity statusEntity = getConnectionStatus(connection.getId());
+            return statusEntity.getConnectionStatus().getAggregateSnapshot().getFlowFilesQueued() == 20;
+        });
+
+        final ClusterEntity clusterEntity = getNifiClient().getControllerClient().getNodes();
+        final Collection<NodeDTO> nodes = clusterEntity.getCluster().getNodes();
+
+        // Do not disconnect the node that the client is pointing out
+        final NodeDTO firstNodeDto = nodes.stream()
+            .filter(nodeDto -> nodeDto.getApiPort() != 5671)
+            .findFirst()
+            .get();
+
+        final String nodeId = firstNodeDto.getNodeId();
+
+        getClientUtil().disconnectNode(nodeId);
+        getClientUtil().offloadNode(nodeId);
+
+        waitFor(this::isNodeOffloaded);
+
+        assertEquals(20, getQueueSize(connection.getId()));
+        assertEquals(20 * 1024 * 1024, getQueueBytes(connection.getId()));
+
+        getClientUtil().connectNode(nodeId);
+        waitForAllNodesConnected();
+
+        // The node that was disconnected will have stopped its processor when it was told to offload. When it joins back into the cluster,
+        // the node will determine that the cluster wants the GenerateFlowFile processor running and as a result start the Processor again. This will
+        // Trigger the processor to then generate another batch of 10 FlowFiles.
+        waitFor(() -> getQueueSize(connection.getId()) == 30);
+        assertEquals(30 * 1024 * 1024, getQueueBytes(connection.getId()));
+    }
+
+    private int getQueueSize(final String connectionId) {
+        final ConnectionStatusEntity statusEntity = getConnectionStatus(connectionId);
+        final ConnectionStatusDTO connectionStatusDto = statusEntity.getConnectionStatus();
+        return connectionStatusDto.getAggregateSnapshot().getFlowFilesQueued().intValue();
+    }
+
+    private long getQueueBytes(final String connectionId) {
+        final ConnectionStatusEntity statusEntity = getConnectionStatus(connectionId);
+        final ConnectionStatusDTO connectionStatusDto = statusEntity.getConnectionStatus();
+        return connectionStatusDto.getAggregateSnapshot().getBytesQueued().longValue();
+    }
+
+
+    private boolean isNodeOffloaded() {
+        final ClusterEntity clusterEntity;
+        try {
+            clusterEntity = getNifiClient().getControllerClient().getNodes();
+        } catch (final Exception e) {
+            e.printStackTrace();
+            return false;
+        }
+
+        final Collection<NodeDTO> nodeDtos = clusterEntity.getCluster().getNodes();
+
+        for (final NodeDTO dto : nodeDtos) {
+            final String status = dto.getStatus();
+            if (status.equalsIgnoreCase("OFFLOADED")) {
+                return true;
+            }
+        }
+
+        return false;
+    }
+
+    private boolean isConnectionDoneLoadBalancing(final String connectionId) {
+        try {
+            final ConnectionEntity connectionEntity = getNifiClient().getConnectionClient().getConnection(connectionId);
+            final String loadBalanceStatus = connectionEntity.getComponent().getLoadBalanceStatus();
+            return ConnectionDTO.LOAD_BALANCE_INACTIVE.equals(loadBalanceStatus);
+        } catch (Exception e) {
+            e.printStackTrace();
+            return false;
+        }
+    }
+
+    private ConnectionStatusEntity getConnectionStatus(final String connectionId) {
+        try {
+            return getNifiClient().getFlowClient().getConnectionStatus(connectionId, true);
+        } catch (final Exception e) {
+            Assert.fail("Failed to obtain connection status");
+            return null;
+        }
+    }
+
+    private boolean isEvenlyDistributed(final ConnectionStatusEntity statusEntity) {
+        final ConnectionStatusDTO connectionStatusDto = statusEntity.getConnectionStatus();
+
+        final LongSummaryStatistics stats = connectionStatusDto.getNodeSnapshots().stream()
+            .map(NodeConnectionStatusSnapshotDTO::getStatusSnapshot)
+            .mapToLong(ConnectionStatusSnapshotDTO::getFlowFilesQueued)
+            .summaryStatistics();
+
+        return stats.getMin() == stats.getMax();
+    }
+
+}
diff --git a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/ControllerServicesClient.java b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/parameters/ClusteredParameterContextIT.java
similarity index 57%
copy from nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/ControllerServicesClient.java
copy to nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/parameters/ClusteredParameterContextIT.java
index af55421..4f994fe 100644
--- a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/ControllerServicesClient.java
+++ b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/parameters/ClusteredParameterContextIT.java
@@ -14,20 +14,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.nifi.toolkit.cli.impl.client.nifi;
+package org.apache.nifi.tests.system.parameters;
 
-import org.apache.nifi.web.api.entity.ControllerServiceEntity;
-import org.apache.nifi.web.api.entity.ControllerServiceRunStatusEntity;
-
-import java.io.IOException;
+import org.apache.nifi.tests.system.SpawnedClusterNiFiInstanceFactory;
+import org.apache.nifi.tests.system.NiFiInstanceFactory;
 
 /**
- * Client for interacting with NiFi's Controller Services Resource.
+ * Repeats all tests in ParameterContextIT but in a clustered mode
  */
-public interface ControllerServicesClient {
-
-    ControllerServiceEntity getControllerService(String id) throws NiFiClientException, IOException;
-
-    ControllerServiceEntity activateControllerService(String id, ControllerServiceRunStatusEntity runStatusEntity) throws NiFiClientException, IOException;
-
+public class ClusteredParameterContextIT extends ParameterContextIT {
+    @Override
+    protected NiFiInstanceFactory getInstanceFactory() {
+        return new SpawnedClusterNiFiInstanceFactory(
+            "src/test/resources/conf/clustered/node1/bootstrap.conf",
+            "src/test/resources/conf/clustered/node2/bootstrap.conf");
+    }
 }
diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/parameters/ParameterContextIT.java b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/parameters/ParameterContextIT.java
new file mode 100644
index 0000000..2221789
--- /dev/null
+++ b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/parameters/ParameterContextIT.java
@@ -0,0 +1,357 @@
+/*
+ * 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.
+ */
+package org.apache.nifi.tests.system.parameters;
+
+import org.apache.nifi.tests.system.NiFiSystemIT;
+import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientException;
+import org.apache.nifi.toolkit.cli.impl.client.nifi.ParamContextClient;
+import org.apache.nifi.web.api.dto.ParameterContextDTO;
+import org.apache.nifi.web.api.dto.ParameterDTO;
+import org.apache.nifi.web.api.dto.ProcessorConfigDTO;
+import org.apache.nifi.web.api.entity.AffectedComponentEntity;
+import org.apache.nifi.web.api.entity.ControllerServiceEntity;
+import org.apache.nifi.web.api.entity.ParameterContextEntity;
+import org.apache.nifi.web.api.entity.ParameterContextUpdateRequestEntity;
+import org.apache.nifi.web.api.entity.ParameterEntity;
+import org.apache.nifi.web.api.entity.ProcessGroupEntity;
+import org.apache.nifi.web.api.entity.ProcessorEntity;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertTrue;
+
+public class ParameterContextIT extends NiFiSystemIT {
+
+    @Test
+    public void testCreateParameterContext() throws NiFiClientException, IOException {
+        final Set<ParameterEntity> parameterEntities = new HashSet<>();
+        parameterEntities.add(createParameterEntity("foo", null, false, "bar"));
+        final ParameterContextEntity entity = createParameterContextEntity(getTestName(), "System Test for verifying creation of Parameter Context", parameterEntities);
+
+        final ParamContextClient paramContextClient = getNifiClient().getParamContextClient();
+        final ParameterContextEntity returned = paramContextClient.createParamContext(entity);
+        assertSingleFooCreation(returned);
+
+        final String contextId = returned.getId();
+        final ParameterContextEntity fetched = paramContextClient.getParamContext(contextId);
+        assertSingleFooCreation(fetched);
+    }
+
+    private void assertSingleFooCreation(final ParameterContextEntity entity) {
+        final ParameterContextDTO returnedDto = entity.getComponent();
+        assertEquals(getTestName(), returnedDto.getName());
+
+        final Set<ParameterEntity> returnedParamEntities = returnedDto.getParameters();
+        assertEquals(1, returnedParamEntities.size());
+        final ParameterDTO returnedParamDto = returnedParamEntities.iterator().next().getParameter();
+        assertEquals("foo", returnedParamDto.getName());
+        assertNull(returnedParamDto.getDescription());
+        assertSame(Boolean.FALSE,returnedParamDto.getSensitive());
+        assertEquals("bar", returnedParamDto.getValue());
+    }
+
+    @Test
+    public void testSensitiveParametersNotReturned() throws NiFiClientException, IOException {
+        final Set<ParameterEntity> parameterEntities = new HashSet<>();
+        parameterEntities.add(createParameterEntity("foo", null, true, "bar"));
+
+        final ParameterContextEntity entity = createParameterContextEntity(getTestName(), null, parameterEntities);
+        final ParamContextClient paramContextClient = getNifiClient().getParamContextClient();
+        final ParameterContextEntity returned = paramContextClient.createParamContext(entity);
+        assertSensitiveParametersNotReturned(returned);
+
+        final String contextId = returned.getId();
+        final ParameterContextEntity fetched = paramContextClient.getParamContext(contextId);
+        assertSensitiveParametersNotReturned(fetched);
+    }
+
+    private void assertSensitiveParametersNotReturned(final ParameterContextEntity entity) {
+        final ParameterContextDTO dto = entity.getComponent();
+        assertEquals(getTestName(), dto.getName());
+
+        final Set<ParameterEntity> returnedParamEntities = dto.getParameters();
+        assertEquals(1, returnedParamEntities.size());
+
+        final ParameterDTO returnedParamDto = returnedParamEntities.iterator().next().getParameter();
+        assertEquals("foo", returnedParamDto.getName());
+        assertNull(returnedParamDto.getDescription());
+        assertSame(Boolean.TRUE,returnedParamDto.getSensitive());
+        assertEquals("********", returnedParamDto.getValue());
+    }
+
+
+    @Test(timeout = 30000)
+    public void testAddingMissingParameterMakesProcessorValid() throws NiFiClientException, IOException, InterruptedException {
+        final ProcessorEntity createdProcessorEntity = createProcessor(TEST_PROCESSORS_PACKAGE + ".CountEvents", NIFI_GROUP_ID, TEST_EXTENSIONS_ARTIFACT_ID, getNiFiVersion());
+        final String processorId = createdProcessorEntity.getId();
+
+        final ProcessorConfigDTO config = createdProcessorEntity.getComponent().getConfig();
+        config.setProperties(Collections.singletonMap("Name", "#{foo}"));
+        getNifiClient().getProcessorClient().updateProcessor(createdProcessorEntity);
+
+        waitForInvalidProcessor(processorId);
+
+        final Set<ParameterEntity> parameters = new HashSet<>();
+        parameters.add(createParameterEntity("foo", null, false, "bar"));
+        final ParameterContextEntity contextEntity = createParameterContextEntity(getTestName(), null, parameters);
+        final ParameterContextEntity createdContextEntity = getNifiClient().getParamContextClient().createParamContext(contextEntity);
+
+        setParameterContext("root", createdContextEntity);
+        waitForValidProcessor(processorId);
+    }
+
+    @Test
+    public void testProcessorStartedAfterLongValidationPeriod() throws NiFiClientException, IOException, InterruptedException {
+        final ParameterContextEntity createdContextEntity = createParameterContext("sleep", "6 secs");
+
+        // Set the Parameter Context on the root Process Group
+        setParameterContext("root", createdContextEntity);
+
+        // Create a Processor and update it to reference Parameter "name"
+        ProcessorEntity processorEntity = createProcessor(TEST_PROCESSORS_PACKAGE + ".Sleep", NIFI_GROUP_ID, TEST_EXTENSIONS_ARTIFACT_ID, getNiFiVersion());
+        final String processorId = processorEntity.getId();
+
+        // Update processor to reference Parameter "name"
+        final ProcessorConfigDTO config = processorEntity.getComponent().getConfig();
+        config.setProperties(Collections.singletonMap("Validate Sleep Time", "#{sleep}"));
+        config.setAutoTerminatedRelationships(Collections.singleton("success"));
+        getNifiClient().getProcessorClient().updateProcessor(processorEntity);
+
+        waitForValidProcessor(processorId);
+
+        // Start Processors
+        getNifiClient().getProcessorClient().startProcessor(processorId, processorEntity.getRevision().getClientId(), 1);
+
+        try {
+            // Update Parameter Context to a long validation time.
+            final ParameterContextUpdateRequestEntity updateRequestEntity = updateParameterContext(createdContextEntity, "sleep", "6 sec");
+
+            final Set<AffectedComponentEntity> affectedComponents = updateRequestEntity.getRequest().getReferencingComponents();
+            assertEquals(1, affectedComponents.size());
+            final String affectedComponentId = affectedComponents.iterator().next().getId();
+            assertEquals(processorId, affectedComponentId);
+
+            getClientUtil().waitForParameterContextRequestToComplete(createdContextEntity.getId(), updateRequestEntity.getRequest().getRequestId());
+
+            waitForRunningProcessor(processorId);
+        } finally {
+            // Ensure that we stop the processor so that other tests are allowed to change the Parameter Context, etc.
+            getNifiClient().getProcessorClient().stopProcessor(processorId, processorEntity.getRevision().getClientId(), 2);
+            waitForStoppedProcessor(processorId);
+
+            getNifiClient().getProcessorClient().deleteProcessor(processorId, processorEntity.getRevision().getClientId(), 3);
+        }
+    }
+
+    @Test
+    public void testProcessorRestartedAfterLongDependentServiceValidationPeriod() throws NiFiClientException, IOException, InterruptedException {
+        final ParameterContextEntity createdContextEntity = createParameterContext("sleep", "0 secs");
+
+        // Set the Parameter Context on the root Process Group
+        setParameterContext("root", createdContextEntity);
+
+        final ControllerServiceEntity serviceEntity = createControllerService(TEST_CS_PACKAGE + ".StandardSleepService", "root", NIFI_GROUP_ID, TEST_EXTENSIONS_ARTIFACT_ID, getNiFiVersion());
+        final String serviceId = serviceEntity.getId();
+
+        // Set service's sleep time to the parameter.
+        serviceEntity.getComponent().setProperties(Collections.singletonMap("Validate Sleep Time", "#{sleep}"));
+        getNifiClient().getControllerServicesClient().updateControllerService(serviceEntity);
+        getClientUtil().enableControllerService(serviceEntity);
+
+        try {
+            // Create a Processor
+            ProcessorEntity processorEntity = createProcessor(TEST_PROCESSORS_PACKAGE + ".Sleep", NIFI_GROUP_ID, TEST_EXTENSIONS_ARTIFACT_ID, getNiFiVersion());
+            final String processorId = processorEntity.getId();
+
+            processorEntity.getComponent().getConfig().setProperties(Collections.singletonMap("Sleep Service", serviceId));
+            processorEntity.getComponent().getConfig().setAutoTerminatedRelationships(Collections.singleton("success"));
+
+            getNifiClient().getProcessorClient().updateProcessor(processorEntity);
+            getNifiClient().getProcessorClient().startProcessor(processorId, processorEntity.getRevision().getClientId(), 1L);
+
+            try {
+                final ParameterContextUpdateRequestEntity requestEntity = updateParameterContext(createdContextEntity, "sleep", "6 secs");
+                final Set<AffectedComponentEntity> affectedComponentEntities = requestEntity.getRequest().getReferencingComponents();
+                assertEquals(2, affectedComponentEntities.size());
+
+                final Set<String> affectedComponentIds = affectedComponentEntities.stream()
+                    .map(AffectedComponentEntity::getId)
+                    .collect(Collectors.toSet());
+
+                assertTrue(affectedComponentIds.contains(serviceId));
+                assertTrue(affectedComponentIds.contains(processorId));
+
+                waitForRunningProcessor(processorId);
+            } finally {
+                getNifiClient().getProcessorClient().stopProcessor(processorId, processorEntity.getRevision().getClientId(), 1L);
+                waitForStoppedProcessor(processorId);
+                getNifiClient().getProcessorClient().deleteProcessor(processorId, processorEntity.getRevision().getClientId(), 3);
+            }
+        } finally {
+            getClientUtil().disableControllerService(serviceEntity);
+            getNifiClient().getControllerServicesClient().deleteControllerService(serviceEntity);
+        }
+    }
+
+
+    @Test
+    public void testProcessorRestartedWhenParameterChanged() throws NiFiClientException, IOException, InterruptedException {
+        testProcessorRestartedWhenParameterChanged("#{name}");
+    }
+
+    @Test
+    public void testProcessorRestartedWhenParameterChangedWhenReferencedThroughEL() throws NiFiClientException, IOException, InterruptedException {
+        testProcessorRestartedWhenParameterChanged("${'hello':equals(#{name})}");
+    }
+
+    private void testProcessorRestartedWhenParameterChanged(final String propertyValue) throws NiFiClientException, IOException, InterruptedException {
+        final Set<ParameterEntity> parameters = new HashSet<>();
+        parameters.add(createParameterEntity("name", null, false, "bar"));
+        final ParameterContextEntity contextEntity = createParameterContextEntity(getTestName(), null, parameters);
+        final ParameterContextEntity createdContextEntity = getNifiClient().getParamContextClient().createParamContext(contextEntity);
+
+        // Set the Parameter Context on the root Process Group
+        setParameterContext("root", createdContextEntity);
+
+        // Create a Processor and update it to reference Parameter "name"
+        ProcessorEntity processorEntity = createProcessor(TEST_PROCESSORS_PACKAGE + ".CountEvents", NIFI_GROUP_ID, TEST_EXTENSIONS_ARTIFACT_ID, getNiFiVersion());
+        final String processorId = processorEntity.getId();
+
+        // Update processor to reference Parameter "name"
+        getClientUtil().updateProcessorProperties(processorEntity, Collections.singletonMap("Name", propertyValue));
+
+        waitForValidProcessor(processorId);
+
+        // Create another processor, and start it. We will not reference any Parameters with this one.
+        final ProcessorEntity secondProcessorEntity = createProcessor(TEST_PROCESSORS_PACKAGE + ".CountEvents", NIFI_GROUP_ID, TEST_EXTENSIONS_ARTIFACT_ID, getNiFiVersion());
+
+        // Start Processors
+        getNifiClient().getProcessorClient().startProcessor(processorEntity.getId(), processorEntity.getRevision().getClientId(), 1L);
+        getNifiClient().getProcessorClient().startProcessor(secondProcessorEntity.getId(), secondProcessorEntity.getRevision().getClientId(), 1L);
+
+        Map<String, Long> counterValues = waitForCounter(processorEntity.getId(), "Scheduled", getNumberOfNodes());
+        assertFalse(counterValues.containsKey("Stopped"));
+
+        final Set<ParameterEntity> createdParameters = createdContextEntity.getComponent().getParameters();
+        createdParameters.clear();
+        createdParameters.add(createParameterEntity("name", "Changed Value from bar to baz", false, "baz"));
+        final ParameterContextUpdateRequestEntity updateRequestEntity = getNifiClient().getParamContextClient().updateParamContext(createdContextEntity);
+        final String requestId = updateRequestEntity.getRequest().getRequestId();
+
+        // Ensure that the Processor is the only Affected Component.
+        final Set<AffectedComponentEntity> affectedComponents = updateRequestEntity.getRequest().getReferencingComponents();
+        assertEquals(1, affectedComponents.size());
+        final AffectedComponentEntity affectedComponentEntity = affectedComponents.iterator().next();
+        assertEquals(processorEntity.getId(), affectedComponentEntity.getId());
+
+        // Wait for the update to complete
+        getClientUtil().waitForParameterContextRequestToComplete(createdContextEntity.getId(), requestId);
+        // Delete the update request
+        getNifiClient().getParamContextClient().deleteParamContextUpdateRequest(createdContextEntity.getId(), requestId);
+
+        // Ensure that the Processor is running
+        processorEntity = getNifiClient().getProcessorClient().getProcessor(processorId);
+        assertEquals("RUNNING", processorEntity.getComponent().getState());
+
+        // Ensure that it has been stopped once and started twice (i.e., it has been restarted). The counters may not immediately
+        // reflect that the Processor has been scheduled twice, depending on timing, so loop while waiting for this to happen.
+        counterValues = getCountersAsMap(processorEntity.getId());
+        assertEquals(getNumberOfNodes(), counterValues.get("Stopped").longValue());
+
+        waitForCounter(processorEntity.getId(), "Scheduled", getNumberOfNodes() * 2);
+
+        // Ensure that the other Processor has been scheduled only once and not stopped.
+        counterValues = getCountersAsMap(secondProcessorEntity.getId());
+        assertFalse(counterValues.containsKey("Stopped"));
+        assertEquals(getNumberOfNodes(), counterValues.get("Scheduled").longValue());
+    }
+
+
+    private Map<String, Long> waitForCounter(final String context, final String counterName, final long expectedValue) throws NiFiClientException, IOException, InterruptedException {
+        return getClientUtil().waitForCounter(context, counterName, expectedValue);
+    }
+
+    private Map<String, Long> getCountersAsMap(final String processorId) throws NiFiClientException, IOException {
+        return getClientUtil().getCountersAsMap(processorId);
+    }
+
+    private ProcessorEntity createProcessor(final String type, final String groupId, final String artifactId, final String version) throws NiFiClientException, IOException {
+        return getClientUtil().createProcessor(type, groupId, artifactId, version);
+    }
+
+    public ControllerServiceEntity createControllerService(final String type, final String processGroupId, final String bundleGroupId, final String artifactId, final String version)
+        throws NiFiClientException, IOException {
+
+        return getClientUtil().createControllerService(type, processGroupId, bundleGroupId, artifactId, version);
+    }
+
+    public ParameterEntity createParameterEntity(final String name, final String description, final boolean sensitive, final String value) {
+        return getClientUtil().createParameterEntity(name, description, sensitive, value);
+    }
+
+    public ParameterContextEntity createParameterContextEntity(final String name, final String description, final Set<ParameterEntity> parameters) {
+        return getClientUtil().createParameterContextEntity(name, description, parameters);
+    }
+
+    private ProcessGroupEntity setParameterContext(final String groupId, final ParameterContextEntity parameterContext) throws NiFiClientException, IOException {
+        return getClientUtil().setParameterContext(groupId, parameterContext);
+    }
+
+    public ParameterContextEntity createParameterContext(final String parameterName, final String parameterValue) throws NiFiClientException, IOException {
+        return createParameterContext(Collections.singletonMap(parameterName, parameterValue));
+    }
+
+    public ParameterContextEntity createParameterContext(final Map<String, String> parameters) throws NiFiClientException, IOException {
+        return getClientUtil().createParameterContext(getTestName(), parameters);
+    }
+
+    public ParameterContextUpdateRequestEntity updateParameterContext(final ParameterContextEntity existingEntity, final String paramName, final String paramValue)
+                throws NiFiClientException, IOException {
+        return updateParameterContext(existingEntity, Collections.singletonMap(paramName, paramValue));
+    }
+
+    public ParameterContextUpdateRequestEntity updateParameterContext(final ParameterContextEntity existingEntity, final Map<String, String> parameters) throws NiFiClientException, IOException {
+        return getClientUtil().updateParameterContext(existingEntity, parameters);
+    }
+
+
+    private void waitForValidProcessor(String id) throws InterruptedException, IOException, NiFiClientException {
+        getClientUtil().waitForValidProcessor(id);
+    }
+
+    private void waitForInvalidProcessor(String id) throws NiFiClientException, IOException, InterruptedException {
+        getClientUtil().waitForInvalidProcessor(id);
+    }
+
+    private void waitForRunningProcessor(final String processorId) throws InterruptedException, IOException, NiFiClientException {
+        getClientUtil().waitForRunningProcessor(processorId);
+    }
+
+    private void waitForStoppedProcessor(final String processorId) throws InterruptedException, IOException, NiFiClientException {
+        getClientUtil().waitForStoppedProcessor(processorId);
+    }
+}
diff --git a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/ControllerServicesClient.java b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/rpg/ClusteredRemoteProcessGroupIT.java
similarity index 57%
copy from nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/ControllerServicesClient.java
copy to nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/rpg/ClusteredRemoteProcessGroupIT.java
index af55421..421d41b 100644
--- a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/ControllerServicesClient.java
+++ b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/rpg/ClusteredRemoteProcessGroupIT.java
@@ -14,20 +14,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.nifi.toolkit.cli.impl.client.nifi;
+package org.apache.nifi.tests.system.rpg;
 
-import org.apache.nifi.web.api.entity.ControllerServiceEntity;
-import org.apache.nifi.web.api.entity.ControllerServiceRunStatusEntity;
+import org.apache.nifi.tests.system.NiFiInstanceFactory;
+import org.apache.nifi.tests.system.SpawnedClusterNiFiInstanceFactory;
 
-import java.io.IOException;
-
-/**
- * Client for interacting with NiFi's Controller Services Resource.
- */
-public interface ControllerServicesClient {
-
-    ControllerServiceEntity getControllerService(String id) throws NiFiClientException, IOException;
-
-    ControllerServiceEntity activateControllerService(String id, ControllerServiceRunStatusEntity runStatusEntity) throws NiFiClientException, IOException;
+public class ClusteredRemoteProcessGroupIT extends RemoteProcessGroupIT {
+    @Override
+    protected NiFiInstanceFactory getInstanceFactory() {
+        return new SpawnedClusterNiFiInstanceFactory(
+            "src/test/resources/conf/clustered/node1/bootstrap.conf",
+            "src/test/resources/conf/clustered/node2/bootstrap.conf");
+    }
 
 }
diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/rpg/RemoteProcessGroupIT.java b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/rpg/RemoteProcessGroupIT.java
new file mode 100644
index 0000000..4bc76b6
--- /dev/null
+++ b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/rpg/RemoteProcessGroupIT.java
@@ -0,0 +1,113 @@
+/*
+ * 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.
+ */
+package org.apache.nifi.tests.system.rpg;
+
+import org.apache.nifi.remote.protocol.SiteToSiteTransportProtocol;
+import org.apache.nifi.tests.system.NiFiClientUtil;
+import org.apache.nifi.tests.system.NiFiSystemIT;
+import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientException;
+import org.apache.nifi.web.api.dto.ConnectableDTO;
+import org.apache.nifi.web.api.dto.RemoteProcessGroupPortDTO;
+import org.apache.nifi.web.api.entity.ConnectionEntity;
+import org.apache.nifi.web.api.entity.PortEntity;
+import org.apache.nifi.web.api.entity.ProcessorEntity;
+import org.apache.nifi.web.api.entity.RemoteProcessGroupEntity;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Set;
+
+public class RemoteProcessGroupIT extends NiFiSystemIT {
+
+    @Test
+    public void testRPGBackToSelfHttp() throws NiFiClientException, IOException, InterruptedException {
+        testRPGBackToSelf(SiteToSiteTransportProtocol.HTTP, "HttpIn");
+    }
+
+    @Test
+    public void testRPGBackToSelfRaw() throws NiFiClientException, IOException, InterruptedException {
+        testRPGBackToSelf(SiteToSiteTransportProtocol.RAW, "RawIn");
+    }
+
+
+    protected void testRPGBackToSelf(final SiteToSiteTransportProtocol protocol, final String portName) throws NiFiClientException, IOException, InterruptedException {
+        final NiFiClientUtil util = getClientUtil();
+
+        // Create a flow that is InputPort -> CountEvents
+        final PortEntity port = util.createRemoteInputPort("root", portName);
+        final ProcessorEntity count = getClientUtil().createProcessor("CountEvents");
+        util.setAutoTerminatedRelationships(count, "success");
+
+        // Create a flow that is GenerateFlowFile -> RPG, connected to the input port
+        final ProcessorEntity generateFlowFile = getClientUtil().createProcessor("GenerateFlowFile");
+        RemoteProcessGroupEntity rpg = getClientUtil().createRPG("root", protocol);
+
+        util.updateProcessorProperties(generateFlowFile, Collections.singletonMap("File Size", "1 KB"));
+        util.updateProcessorProperties(generateFlowFile, Collections.singletonMap("Batch Size", "3"));
+        util.updateProcessorSchedulingPeriod(generateFlowFile, "10 min");
+
+        final String rpgId = rpg.getId();
+
+        // Wait for the port to become available. We have to check for the specific port ID because otherwise,
+        // the RPG may have an old Port ID cached, since we are running an HTTP-based test and a RAW-based test
+        waitFor(() -> {
+            try {
+                final RemoteProcessGroupEntity entity = getNifiClient().getRemoteProcessGroupClient().getRemoteProcessGroup(rpgId);
+                final Set<RemoteProcessGroupPortDTO> ports = entity.getComponent().getContents().getInputPorts();
+                if (ports.isEmpty()) {
+                    return false;
+                }
+
+                for (final RemoteProcessGroupPortDTO dto : ports) {
+                    if (dto.getTargetId().equals(port.getId())) {
+                        return true;
+                    }
+                }
+
+                return false;
+            } catch (Exception e) {
+                e.printStackTrace();
+                Assert.fail("Could not retrieve RPG with ID " + rpgId);
+                return false;
+            }
+        });
+
+        rpg = getNifiClient().getRemoteProcessGroupClient().getRemoteProcessGroup(rpg.getId());
+        final String rpgPortId = rpg.getComponent().getContents().getInputPorts().stream()
+            .filter(dto -> dto.getTargetId().equals(port.getId()))
+            .findFirst() // find the port with the desired ID
+            .get() // get the Port
+            .getId(); // get the Port's ID
+
+        final ConnectableDTO destination = new ConnectableDTO();
+        destination.setId(rpgPortId);
+        destination.setGroupId(rpg.getId());
+        destination.setType("REMOTE_INPUT_PORT");
+
+        final ConnectionEntity generateToRPG = getClientUtil().createConnection(util.createConnectableDTO(generateFlowFile), destination, "success");
+        final ConnectionEntity portToCount = getClientUtil().createConnection(util.createConnectableDTO(port), util.createConnectableDTO(count), "");
+
+        getNifiClient().getInputPortClient().startInputPort(port);
+        getNifiClient().getProcessorClient().startProcessor(generateFlowFile);
+        getNifiClient().getRemoteProcessGroupClient().startTransmitting(rpg);
+
+        waitFor(() -> util.getQueueSize(generateToRPG.getId()).getObjectCount() == 0);
+        waitFor(() -> util.getQueueSize(portToCount.getId()).getObjectCount() == 3 * getNumberOfNodes());
+    }
+}
diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/resources/conf/clustered/node1/bootstrap.conf b/nifi-system-tests/nifi-system-test-suite/src/test/resources/conf/clustered/node1/bootstrap.conf
new file mode 100644
index 0000000..297108b
--- /dev/null
+++ b/nifi-system-tests/nifi-system-test-suite/src/test/resources/conf/clustered/node1/bootstrap.conf
@@ -0,0 +1,33 @@
+# 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.
+
+# Configure where NiFi's lib and conf directories live
+lib.dir=../nifi-lib-assembly/lib
+conf.dir=./conf
+working.dir=./target/node1
+#java=/Library/Java/JavaVirtualMachines/jdk1.8.0_60.jdk/Contents/Home/bin/java
+
+# How long to wait after telling NiFi to shutdown before explicitly killing the Process
+graceful.shutdown.seconds=20
+
+# JVM memory settings
+java.arg.2= -Xms128m
+java.arg.3=-Xmx128m
+
+java.arg.14=-Djava.awt.headless=true
+
+#java.arg.debug=-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=8002
+
+nifi.bootstrap.sensitive.key=
\ No newline at end of file
diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/resources/conf/clustered/node1/logback.xml b/nifi-system-tests/nifi-system-test-suite/src/test/resources/conf/clustered/node1/logback.xml
new file mode 100644
index 0000000..0a91790
--- /dev/null
+++ b/nifi-system-tests/nifi-system-test-suite/src/test/resources/conf/clustered/node1/logback.xml
@@ -0,0 +1,177 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+
+<configuration scan="true" scanPeriod="30 seconds">
+    <contextListener class="ch.qos.logback.classic.jul.LevelChangePropagator">
+        <resetJUL>true</resetJUL>
+    </contextListener>
+
+    <appender name="APP_FILE" class="ch.qos.logback.core.rolling.RollingFileAppender">
+        <file>${org.apache.nifi.bootstrap.config.log.dir}/nifi-app.log</file>
+        <rollingPolicy class="ch.qos.logback.core.rolling.SizeAndTimeBasedRollingPolicy">
+            <!--
+              For daily rollover, use 'app_%d.log'.
+              For hourly rollover, use 'app_%d{yyyy-MM-dd_HH}.log'.
+              To GZIP rolled files, replace '.log' with '.log.gz'.
+              To ZIP rolled files, replace '.log' with '.log.zip'.
+            -->
+            <fileNamePattern>${org.apache.nifi.bootstrap.config.log.dir}/nifi-app_%d{yyyy-MM-dd_HH}.%i.log</fileNamePattern>
+            <maxFileSize>100MB</maxFileSize>
+            <!-- keep 30 log files worth of history -->
+            <maxHistory>30</maxHistory>
+        </rollingPolicy>
+        <immediateFlush>true</immediateFlush>
+        <encoder class="ch.qos.logback.classic.encoder.PatternLayoutEncoder">
+            <pattern>%date %level [%thread] %logger{40} %msg%n</pattern>
+        </encoder>
+    </appender>
+
+    <appender name="USER_FILE" class="ch.qos.logback.core.rolling.RollingFileAppender">
+        <file>${org.apache.nifi.bootstrap.config.log.dir}/nifi-user.log</file>
+        <rollingPolicy class="ch.qos.logback.core.rolling.TimeBasedRollingPolicy">
+            <!--
+              For daily rollover, use 'user_%d.log'.
+              For hourly rollover, use 'user_%d{yyyy-MM-dd_HH}.log'.
+              To GZIP rolled files, replace '.log' with '.log.gz'.
+              To ZIP rolled files, replace '.log' with '.log.zip'.
+            -->
+            <fileNamePattern>${org.apache.nifi.bootstrap.config.log.dir}/nifi-user_%d.log</fileNamePattern>
+            <!-- keep 30 log files worth of history -->
+            <maxHistory>30</maxHistory>
+        </rollingPolicy>
+        <encoder class="ch.qos.logback.classic.encoder.PatternLayoutEncoder">
+            <pattern>%date %level [%thread] %logger{40} %msg%n</pattern>
+        </encoder>
+    </appender>
+
+    <appender name="BOOTSTRAP_FILE" class="ch.qos.logback.core.rolling.RollingFileAppender">
+        <file>${org.apache.nifi.bootstrap.config.log.dir}/nifi-bootstrap.log</file>
+        <rollingPolicy class="ch.qos.logback.core.rolling.TimeBasedRollingPolicy">
+            <!--
+              For daily rollover, use 'user_%d.log'.
+              For hourly rollover, use 'user_%d{yyyy-MM-dd_HH}.log'.
+              To GZIP rolled files, replace '.log' with '.log.gz'.
+              To ZIP rolled files, replace '.log' with '.log.zip'.
+            -->
+            <fileNamePattern>${org.apache.nifi.bootstrap.config.log.dir}/nifi-bootstrap_%d.log</fileNamePattern>
+            <!-- keep 5 log files worth of history -->
+            <maxHistory>5</maxHistory>
+        </rollingPolicy>
+        <encoder class="ch.qos.logback.classic.encoder.PatternLayoutEncoder">
+            <pattern>%date %level [%thread] %logger{40} %msg%n</pattern>
+        </encoder>
+    </appender>
+
+    <appender name="CONSOLE" class="ch.qos.logback.core.ConsoleAppender">
+        <encoder class="ch.qos.logback.classic.encoder.PatternLayoutEncoder">
+            <pattern>%date %level [%thread] %logger{40} %msg%n</pattern>
+        </encoder>
+    </appender>
+
+    <!-- valid logging levels: TRACE, DEBUG, INFO, WARN, ERROR -->
+    <logger name="org.apache.nifi" level="INFO"/>
+    <logger name="org.apache.nifi.processors" level="WARN"/>
+    <logger name="org.apache.nifi.processors.standard.LogAttribute" level="INFO"/>
+    <logger name="org.apache.nifi.processors.standard.LogMessage" level="INFO"/>
+    <logger name="org.apache.nifi.controller.repository.StandardProcessSession" level="WARN" />
+
+
+    <logger name="org.apache.zookeeper.ClientCnxn" level="ERROR" />
+    <logger name="org.apache.zookeeper.server.NIOServerCnxn" level="ERROR" />
+    <logger name="org.apache.zookeeper.server.NIOServerCnxnFactory" level="ERROR" />
+    <logger name="org.apache.zookeeper.server.quorum" level="ERROR" />
+    <logger name="org.apache.zookeeper.ZooKeeper" level="ERROR" />
+    <logger name="org.apache.zookeeper.server.PrepRequestProcessor" level="ERROR" />
+
+    <logger name="org.apache.calcite.runtime.CalciteException" level="OFF" />
+
+    <logger name="org.apache.curator.framework.recipes.leader.LeaderSelector" level="OFF" />
+    <logger name="org.apache.curator.ConnectionState" level="OFF" />
+
+    <!-- Logger for managing logging statements for nifi clusters. -->
+    <logger name="org.apache.nifi.cluster" level="INFO"/>
+
+    <!-- Logger for logging HTTP requests received by the web server. -->
+    <logger name="org.apache.nifi.server.JettyServer" level="INFO"/>
+
+    <!-- Logger for managing logging statements for jetty -->
+    <logger name="org.eclipse.jetty" level="INFO"/>
+
+    <!-- Suppress non-error messages due to excessive logging by class or library -->
+    <logger name="org.springframework" level="ERROR"/>
+
+    <!-- Suppress non-error messages due to known warning about redundant path annotation (NIFI-574) -->
+    <logger name="org.glassfish.jersey.internal.Errors" level="ERROR"/>
+
+    <!-- Suppress non-error messages due to Jetty AnnotationParser emitting a large amount of WARNS. Issue described in NIFI-5479. -->
+    <logger name="org.eclipse.jetty.annotations.AnnotationParser" level="ERROR"/>
+
+    <!-- Suppress non-error messages from SSHJ which was emitting large amounts of INFO logs by default -->
+    <logger name="net.schmizz.sshj" level="WARN" />
+    <logger name="com.hierynomus.sshj" level="WARN" />
+
+    <!--
+        Logger for capturing user events. We do not want to propagate these
+        log events to the root logger. These messages are only sent to the
+        user-log appender.
+    -->
+    <logger name="org.apache.nifi.web.security" level="INFO" additivity="false">
+        <appender-ref ref="USER_FILE"/>
+    </logger>
+    <logger name="org.apache.nifi.web.api.config" level="INFO" additivity="false">
+        <appender-ref ref="USER_FILE"/>
+    </logger>
+    <logger name="org.apache.nifi.authorization" level="INFO" additivity="false">
+        <appender-ref ref="USER_FILE"/>
+    </logger>
+    <logger name="org.apache.nifi.cluster.authorization" level="INFO" additivity="false">
+        <appender-ref ref="USER_FILE"/>
+    </logger>
+    <logger name="org.apache.nifi.web.filter.RequestLogger" level="INFO" additivity="false">
+        <appender-ref ref="USER_FILE"/>
+    </logger>
+    <logger name="org.apache.nifi.web.api.AccessResource" level="INFO" additivity="false">
+        <appender-ref ref="USER_FILE"/>
+    </logger>
+
+
+    <!--
+        Logger for capturing Bootstrap logs and NiFi's standard error and standard out.
+    -->
+    <logger name="org.apache.nifi.bootstrap" level="INFO" additivity="false">
+        <appender-ref ref="BOOTSTRAP_FILE" />
+    </logger>
+    <logger name="org.apache.nifi.bootstrap.Command" level="INFO" additivity="false">
+        <appender-ref ref="CONSOLE" />
+        <appender-ref ref="BOOTSTRAP_FILE" />
+    </logger>
+
+    <!-- Everything written to NiFi's Standard Out will be logged with the logger org.apache.nifi.StdOut at INFO level -->
+    <logger name="org.apache.nifi.StdOut" level="INFO" additivity="false">
+        <appender-ref ref="BOOTSTRAP_FILE" />
+    </logger>
+
+    <!-- Everything written to NiFi's Standard Error will be logged with the logger org.apache.nifi.StdErr at ERROR level -->
+    <logger name="org.apache.nifi.StdErr" level="ERROR" additivity="false">
+        <appender-ref ref="BOOTSTRAP_FILE" />
+    </logger>
+
+
+    <root level="INFO">
+        <appender-ref ref="APP_FILE"/>
+    </root>
+
+</configuration>
\ No newline at end of file
diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/resources/conf/clustered/node1/nifi.properties b/nifi-system-tests/nifi-system-test-suite/src/test/resources/conf/clustered/node1/nifi.properties
new file mode 100644
index 0000000..d5d79ae
--- /dev/null
+++ b/nifi-system-tests/nifi-system-test-suite/src/test/resources/conf/clustered/node1/nifi.properties
@@ -0,0 +1,256 @@
+# 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.
+
+# Core Properties #
+nifi.flow.configuration.file=./conf/flow.xml.gz
+nifi.flow.configuration.archive.enabled=true
+nifi.flow.configuration.archive.dir=./conf/archive/
+nifi.flow.configuration.archive.max.time=30 days
+nifi.flow.configuration.archive.max.storage=500 MB
+nifi.flow.configuration.archive.max.count=
+nifi.flowcontroller.autoResumeState=true
+nifi.flowcontroller.graceful.shutdown.period=10 sec
+nifi.flowservice.writedelay.interval=500 ms
+nifi.administrative.yield.duration=100 millis
+# If a component has no work to do (is "bored"), how long should we wait before checking again for work?
+nifi.bored.yield.duration=10 millis
+nifi.queue.backpressure.count=10000
+nifi.queue.backpressure.size=1 GB
+
+nifi.authorizer.configuration.file=./conf/authorizers.xml
+nifi.login.identity.provider.configuration.file=./conf/login-identity-providers.xml
+nifi.templates.directory=./conf/templates
+nifi.ui.banner.text=
+nifi.ui.autorefresh.interval=30 sec
+nifi.nar.library.directory=./lib
+nifi.nar.library.autoload.directory=./extensions
+nifi.nar.working.directory=./work/nar/
+nifi.documentation.working.directory=./work/docs/components
+
+####################
+# State Management #
+####################
+nifi.state.management.configuration.file=./conf/state-management.xml
+# The ID of the local state provider
+nifi.state.management.provider.local=local-provider
+# The ID of the cluster-wide state provider. This will be ignored if NiFi is not clustered but must be populated if running in a cluster.
+nifi.state.management.provider.cluster=zk-provider
+# Specifies whether or not this instance of NiFi should run an embedded ZooKeeper server
+nifi.state.management.embedded.zookeeper.start=true
+# Properties file that provides the ZooKeeper properties to use if <nifi.state.management.embedded.zookeeper.start> is set to true
+nifi.state.management.embedded.zookeeper.properties=./conf/zookeeper.properties
+
+
+# H2 Settings
+nifi.database.directory=./database_repository
+nifi.h2.url.append=;LOCK_TIMEOUT=25000;WRITE_DELAY=0;AUTO_SERVER=FALSE
+
+# FlowFile Repository
+nifi.flowfile.repository.implementation=org.apache.nifi.controller.repository.WriteAheadFlowFileRepository
+nifi.flowfile.repository.wal.implementation=org.apache.nifi.wali.SequentialAccessWriteAheadLog
+nifi.flowfile.repository.directory=./flowfile_repository
+nifi.flowfile.repository.partitions=256
+nifi.flowfile.repository.checkpoint.interval=5 mins
+nifi.flowfile.repository.always.sync=false
+
+nifi.swap.manager.implementation=org.apache.nifi.controller.FileSystemSwapManager
+nifi.queue.swap.threshold=20000
+nifi.swap.in.period=5 sec
+nifi.swap.in.threads=1
+nifi.swap.out.period=5 sec
+nifi.swap.out.threads=4
+
+# Content Repository
+nifi.content.repository.implementation=org.apache.nifi.controller.repository.FileSystemRepository
+nifi.content.claim.max.appendable.size=1 MB
+nifi.content.claim.max.flow.files=100
+nifi.content.repository.directory.default=./content_repository
+nifi.content.repository.archive.max.retention.period=12 hours
+nifi.content.repository.archive.max.usage.percentage=50%
+nifi.content.repository.archive.enabled=true
+nifi.content.repository.always.sync=false
+nifi.content.viewer.url=../nifi-content-viewer/
+
+# Provenance Repository Properties
+nifi.provenance.repository.implementation=org.apache.nifi.provenance.WriteAheadProvenanceRepository
+nifi.provenance.repository.debug.frequency=1_000_000
+nifi.provenance.repository.encryption.key.provider.implementation=
+nifi.provenance.repository.encryption.key.provider.location=
+nifi.provenance.repository.encryption.key.id=
+nifi.provenance.repository.encryption.key=
+
+# Persistent Provenance Repository Properties
+nifi.provenance.repository.directory.default=./provenance_repository
+nifi.provenance.repository.max.storage.time=24 hours
+nifi.provenance.repository.max.storage.size=1 GB
+nifi.provenance.repository.rollover.time=30 secs
+nifi.provenance.repository.rollover.size=100 MB
+nifi.provenance.repository.query.threads=2
+nifi.provenance.repository.index.threads=2
+nifi.provenance.repository.compress.on.rollover=true
+nifi.provenance.repository.always.sync=false
+# Comma-separated list of fields. Fields that are not indexed will not be searchable. Valid fields are:
+# EventType, FlowFileUUID, Filename, TransitURI, ProcessorID, AlternateIdentifierURI, Relationship, Details
+nifi.provenance.repository.indexed.fields=EventType, FlowFileUUID, Filename, ProcessorID, Relationship
+# FlowFile Attributes that should be indexed and made searchable.  Some examples to consider are filename, uuid, mime.type
+nifi.provenance.repository.indexed.attributes=
+# Large values for the shard size will result in more Java heap usage when searching the Provenance Repository
+# but should provide better performance
+nifi.provenance.repository.index.shard.size=500 MB
+# Indicates the maximum length that a FlowFile attribute can be when retrieving a Provenance Event from
+# the repository. If the length of any attribute exceeds this value, it will be truncated when the event is retrieved.
+nifi.provenance.repository.max.attribute.length=65536
+nifi.provenance.repository.concurrent.merge.threads=2
+
+
+# Volatile Provenance Respository Properties
+nifi.provenance.repository.buffer.size=100000
+
+# Component Status Repository
+nifi.components.status.repository.implementation=org.apache.nifi.controller.status.history.VolatileComponentStatusRepository
+nifi.components.status.repository.buffer.size=1440
+nifi.components.status.snapshot.frequency=1 min
+
+# Site to Site properties
+nifi.remote.input.host=
+nifi.remote.input.secure=false
+nifi.remote.input.socket.port=7781
+nifi.remote.input.http.enabled=true
+nifi.remote.input.http.transaction.ttl=30 sec
+nifi.remote.contents.cache.expiration=30 secs
+
+# web properties #
+nifi.web.war.directory=./lib
+nifi.web.http.host=
+nifi.web.http.port=5671
+nifi.web.http.network.interface.default=
+nifi.web.https.host=
+nifi.web.https.port=
+nifi.web.https.network.interface.default=
+nifi.web.jetty.working.directory=./work/jetty
+nifi.web.jetty.threads=200
+nifi.web.max.header.size=16 KB
+nifi.web.proxy.context.path=
+nifi.web.proxy.host=
+
+# security properties #
+nifi.sensitive.props.key=
+nifi.sensitive.props.key.protected=
+nifi.sensitive.props.algorithm=PBEWITHMD5AND256BITAES-CBC-OPENSSL
+nifi.sensitive.props.provider=BC
+nifi.sensitive.props.additional.keys=
+
+nifi.security.keystore=certs/keystore.jks
+nifi.security.keystoreType=JKS
+nifi.security.keystorePasswd=passwordpassword
+nifi.security.keyPasswd=
+nifi.security.truststore=certs/truststore.jks
+nifi.security.truststoreType=JKS
+nifi.security.truststorePasswd=passwordpassword
+nifi.security.user.authorizer=managed-authorizer
+nifi.security.user.login.identity.provider=
+nifi.security.ocsp.responder.url=
+nifi.security.ocsp.responder.certificate=
+
+# OpenId Connect SSO Properties #
+nifi.security.user.oidc.discovery.url=
+nifi.security.user.oidc.connect.timeout=5 secs
+nifi.security.user.oidc.read.timeout=5 secs
+nifi.security.user.oidc.client.id=
+nifi.security.user.oidc.client.secret=
+nifi.security.user.oidc.preferred.jwsalgorithm=
+
+# Apache Knox SSO Properties #
+nifi.security.user.knox.url=
+nifi.security.user.knox.publicKey=
+nifi.security.user.knox.cookieName=hadoop-jwt
+nifi.security.user.knox.audiences=
+
+# Identity Mapping Properties #
+# These properties allow normalizing user identities such that identities coming from different identity providers
+# (certificates, LDAP, Kerberos) can be treated the same internally in NiFi. The following example demonstrates normalizing
+# DNs from certificates and principals from Kerberos into a common identity string:
+#
+# nifi.security.identity.mapping.pattern.dn=^CN=(.*?), OU=(.*?), O=(.*?), L=(.*?), ST=(.*?), C=(.*?)$
+# nifi.security.identity.mapping.value.dn=$1@$2
+# nifi.security.identity.mapping.transform.dn=NONE
+# nifi.security.identity.mapping.pattern.kerb=^(.*?)/instance@(.*?)$
+# nifi.security.identity.mapping.value.kerb=$1@$2
+# nifi.security.identity.mapping.transform.kerb=UPPER
+
+# Group Mapping Properties #
+# These properties allow normalizing group names coming from external sources like LDAP. The following example
+# lowercases any group name.
+#
+# nifi.security.group.mapping.pattern.anygroup=^(.*)$
+# nifi.security.group.mapping.value.anygroup=$1
+# nifi.security.group.mapping.transform.anygroup=LOWER
+
+# cluster common properties (all nodes must have same values) #
+nifi.cluster.protocol.heartbeat.interval=5 sec
+nifi.cluster.protocol.is.secure=false
+
+# cluster node properties (only configure for cluster nodes) #
+nifi.cluster.is.node=true
+nifi.cluster.node.address=
+nifi.cluster.node.protocol.port=48101
+nifi.cluster.node.protocol.threads=10
+nifi.cluster.node.protocol.max.threads=50
+nifi.cluster.node.event.history.size=25
+nifi.cluster.node.connection.timeout=5 sec
+nifi.cluster.node.read.timeout=5 sec
+nifi.cluster.node.max.concurrent.requests=100
+nifi.cluster.firewall.file=
+nifi.cluster.flow.election.max.wait.time=1 secs
+nifi.cluster.flow.election.max.candidates=1
+
+# cluster load balancing properties #
+nifi.cluster.load.balance.host=
+nifi.cluster.load.balance.port=6341
+nifi.cluster.load.balance.connections.per.node=4
+nifi.cluster.load.balance.max.thread.count=8
+nifi.cluster.load.balance.comms.timeout=30 sec
+
+# zookeeper properties, used for cluster management #
+nifi.zookeeper.connect.string=localhost:21818
+nifi.zookeeper.connect.timeout=3 secs
+nifi.zookeeper.session.timeout=3 secs
+nifi.zookeeper.root.node=/nifi
+
+# Zookeeper properties for the authentication scheme used when creating acls on znodes used for cluster management
+# Values supported for nifi.zookeeper.auth.type are "default", which will apply world/anyone rights on znodes
+# and "sasl" which will give rights to the sasl/kerberos identity used to authenticate the nifi node
+# The identity is determined using the value in nifi.kerberos.service.principal and the removeHostFromPrincipal
+# and removeRealmFromPrincipal values (which should align with the kerberos.removeHostFromPrincipal and kerberos.removeRealmFromPrincipal
+# values configured on the zookeeper server).
+nifi.zookeeper.auth.type=
+nifi.zookeeper.kerberos.removeHostFromPrincipal=
+nifi.zookeeper.kerberos.removeRealmFromPrincipal=
+
+# kerberos #
+nifi.kerberos.krb5.file=
+
+# kerberos service principal #
+nifi.kerberos.service.principal=
+nifi.kerberos.service.keytab.location=
+
+# kerberos spnego principal #
+nifi.kerberos.spnego.principal=
+nifi.kerberos.spnego.keytab.location=
+nifi.kerberos.spnego.authentication.expiration=12 hours
+
+# external properties files for variable registry
+# supports a comma delimited list of file locations
+nifi.variable.registry.properties=
\ No newline at end of file
diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/resources/conf/clustered/node1/state-management.xml b/nifi-system-tests/nifi-system-test-suite/src/test/resources/conf/clustered/node1/state-management.xml
new file mode 100644
index 0000000..3dacd13
--- /dev/null
+++ b/nifi-system-tests/nifi-system-test-suite/src/test/resources/conf/clustered/node1/state-management.xml
@@ -0,0 +1,32 @@
+<!--
+  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.
+-->
+<stateManagement>
+    <local-provider>
+        <id>local-provider</id>
+        <class>org.apache.nifi.controller.state.providers.local.WriteAheadLocalStateProvider</class>
+        <property name="Directory">./state/local</property>
+        <property name="Always Sync">false</property>
+        <property name="Partitions">16</property>
+        <property name="Checkpoint Interval">2 mins</property>
+    </local-provider>
+    <cluster-provider>
+        <id>zk-provider</id>
+        <class>org.apache.nifi.controller.state.providers.zookeeper.ZooKeeperStateProvider</class>
+        <property name="Connect String">localhost:21818</property>
+        <property name="Root Node">/nifi-system-test</property>
+        <property name="Session Timeout">30 seconds</property>
+        <property name="Access Control">Open</property>
+    </cluster-provider>
+</stateManagement>
\ No newline at end of file
diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/resources/conf/clustered/node1/zookeeper.properties b/nifi-system-tests/nifi-system-test-suite/src/test/resources/conf/clustered/node1/zookeeper.properties
new file mode 100644
index 0000000..5b4cd02
--- /dev/null
+++ b/nifi-system-tests/nifi-system-test-suite/src/test/resources/conf/clustered/node1/zookeeper.properties
@@ -0,0 +1,45 @@
+#
+#
+# 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.
+#
+#
+#
+
+clientPort=21818
+initLimit=10
+autopurge.purgeInterval=24
+syncLimit=5
+tickTime=2000
+dataDir=./state/zookeeper
+autopurge.snapRetainCount=30
+
+#
+# Specifies the servers that are part of this zookeeper ensemble. For
+# every NiFi instance running an embedded zookeeper, there needs to be
+# a server entry below. For instance:
+#
+# server.1=nifi-node1-hostname:2888:3888
+# server.2=nifi-node2-hostname:2888:3888
+# server.3=nifi-node3-hostname:2888:3888
+#
+# The index of the server corresponds to the myid file that gets created
+# in the dataDir of each node running an embedded zookeeper. See the
+# administration guide for more details.
+#
+
+server.1=localhost:5888:6888;21818
diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/resources/conf/clustered/node2/bootstrap.conf b/nifi-system-tests/nifi-system-test-suite/src/test/resources/conf/clustered/node2/bootstrap.conf
new file mode 100644
index 0000000..b348aa0
--- /dev/null
+++ b/nifi-system-tests/nifi-system-test-suite/src/test/resources/conf/clustered/node2/bootstrap.conf
@@ -0,0 +1,33 @@
+# 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.
+
+# Configure where NiFi's lib and conf directories live
+lib.dir=../nifi-lib-assembly/lib
+conf.dir=./conf
+working.dir=./target/node2
+#java=/Library/Java/JavaVirtualMachines/jdk1.8.0_60.jdk/Contents/Home/bin/java
+
+# How long to wait after telling NiFi to shutdown before explicitly killing the Process
+graceful.shutdown.seconds=20
+
+# JVM memory settings
+java.arg.2= -Xms128m
+java.arg.3=-Xmx128m
+
+java.arg.14=-Djava.awt.headless=true
+
+#java.arg.debug=-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=8003
+
+nifi.bootstrap.sensitive.key=
\ No newline at end of file
diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/resources/conf/clustered/node2/logback.xml b/nifi-system-tests/nifi-system-test-suite/src/test/resources/conf/clustered/node2/logback.xml
new file mode 100644
index 0000000..0a91790
--- /dev/null
+++ b/nifi-system-tests/nifi-system-test-suite/src/test/resources/conf/clustered/node2/logback.xml
@@ -0,0 +1,177 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+
+<configuration scan="true" scanPeriod="30 seconds">
+    <contextListener class="ch.qos.logback.classic.jul.LevelChangePropagator">
+        <resetJUL>true</resetJUL>
+    </contextListener>
+
+    <appender name="APP_FILE" class="ch.qos.logback.core.rolling.RollingFileAppender">
+        <file>${org.apache.nifi.bootstrap.config.log.dir}/nifi-app.log</file>
+        <rollingPolicy class="ch.qos.logback.core.rolling.SizeAndTimeBasedRollingPolicy">
+            <!--
+              For daily rollover, use 'app_%d.log'.
+              For hourly rollover, use 'app_%d{yyyy-MM-dd_HH}.log'.
+              To GZIP rolled files, replace '.log' with '.log.gz'.
+              To ZIP rolled files, replace '.log' with '.log.zip'.
+            -->
+            <fileNamePattern>${org.apache.nifi.bootstrap.config.log.dir}/nifi-app_%d{yyyy-MM-dd_HH}.%i.log</fileNamePattern>
+            <maxFileSize>100MB</maxFileSize>
+            <!-- keep 30 log files worth of history -->
+            <maxHistory>30</maxHistory>
+        </rollingPolicy>
+        <immediateFlush>true</immediateFlush>
+        <encoder class="ch.qos.logback.classic.encoder.PatternLayoutEncoder">
+            <pattern>%date %level [%thread] %logger{40} %msg%n</pattern>
+        </encoder>
+    </appender>
+
+    <appender name="USER_FILE" class="ch.qos.logback.core.rolling.RollingFileAppender">
+        <file>${org.apache.nifi.bootstrap.config.log.dir}/nifi-user.log</file>
+        <rollingPolicy class="ch.qos.logback.core.rolling.TimeBasedRollingPolicy">
+            <!--
+              For daily rollover, use 'user_%d.log'.
+              For hourly rollover, use 'user_%d{yyyy-MM-dd_HH}.log'.
+              To GZIP rolled files, replace '.log' with '.log.gz'.
+              To ZIP rolled files, replace '.log' with '.log.zip'.
+            -->
+            <fileNamePattern>${org.apache.nifi.bootstrap.config.log.dir}/nifi-user_%d.log</fileNamePattern>
+            <!-- keep 30 log files worth of history -->
+            <maxHistory>30</maxHistory>
+        </rollingPolicy>
+        <encoder class="ch.qos.logback.classic.encoder.PatternLayoutEncoder">
+            <pattern>%date %level [%thread] %logger{40} %msg%n</pattern>
+        </encoder>
+    </appender>
+
+    <appender name="BOOTSTRAP_FILE" class="ch.qos.logback.core.rolling.RollingFileAppender">
+        <file>${org.apache.nifi.bootstrap.config.log.dir}/nifi-bootstrap.log</file>
+        <rollingPolicy class="ch.qos.logback.core.rolling.TimeBasedRollingPolicy">
+            <!--
+              For daily rollover, use 'user_%d.log'.
+              For hourly rollover, use 'user_%d{yyyy-MM-dd_HH}.log'.
+              To GZIP rolled files, replace '.log' with '.log.gz'.
+              To ZIP rolled files, replace '.log' with '.log.zip'.
+            -->
+            <fileNamePattern>${org.apache.nifi.bootstrap.config.log.dir}/nifi-bootstrap_%d.log</fileNamePattern>
+            <!-- keep 5 log files worth of history -->
+            <maxHistory>5</maxHistory>
+        </rollingPolicy>
+        <encoder class="ch.qos.logback.classic.encoder.PatternLayoutEncoder">
+            <pattern>%date %level [%thread] %logger{40} %msg%n</pattern>
+        </encoder>
+    </appender>
+
+    <appender name="CONSOLE" class="ch.qos.logback.core.ConsoleAppender">
+        <encoder class="ch.qos.logback.classic.encoder.PatternLayoutEncoder">
+            <pattern>%date %level [%thread] %logger{40} %msg%n</pattern>
+        </encoder>
+    </appender>
+
+    <!-- valid logging levels: TRACE, DEBUG, INFO, WARN, ERROR -->
+    <logger name="org.apache.nifi" level="INFO"/>
+    <logger name="org.apache.nifi.processors" level="WARN"/>
+    <logger name="org.apache.nifi.processors.standard.LogAttribute" level="INFO"/>
+    <logger name="org.apache.nifi.processors.standard.LogMessage" level="INFO"/>
+    <logger name="org.apache.nifi.controller.repository.StandardProcessSession" level="WARN" />
+
+
+    <logger name="org.apache.zookeeper.ClientCnxn" level="ERROR" />
+    <logger name="org.apache.zookeeper.server.NIOServerCnxn" level="ERROR" />
+    <logger name="org.apache.zookeeper.server.NIOServerCnxnFactory" level="ERROR" />
+    <logger name="org.apache.zookeeper.server.quorum" level="ERROR" />
+    <logger name="org.apache.zookeeper.ZooKeeper" level="ERROR" />
+    <logger name="org.apache.zookeeper.server.PrepRequestProcessor" level="ERROR" />
+
+    <logger name="org.apache.calcite.runtime.CalciteException" level="OFF" />
+
+    <logger name="org.apache.curator.framework.recipes.leader.LeaderSelector" level="OFF" />
+    <logger name="org.apache.curator.ConnectionState" level="OFF" />
+
+    <!-- Logger for managing logging statements for nifi clusters. -->
+    <logger name="org.apache.nifi.cluster" level="INFO"/>
+
+    <!-- Logger for logging HTTP requests received by the web server. -->
+    <logger name="org.apache.nifi.server.JettyServer" level="INFO"/>
+
+    <!-- Logger for managing logging statements for jetty -->
+    <logger name="org.eclipse.jetty" level="INFO"/>
+
+    <!-- Suppress non-error messages due to excessive logging by class or library -->
+    <logger name="org.springframework" level="ERROR"/>
+
+    <!-- Suppress non-error messages due to known warning about redundant path annotation (NIFI-574) -->
+    <logger name="org.glassfish.jersey.internal.Errors" level="ERROR"/>
+
+    <!-- Suppress non-error messages due to Jetty AnnotationParser emitting a large amount of WARNS. Issue described in NIFI-5479. -->
+    <logger name="org.eclipse.jetty.annotations.AnnotationParser" level="ERROR"/>
+
+    <!-- Suppress non-error messages from SSHJ which was emitting large amounts of INFO logs by default -->
+    <logger name="net.schmizz.sshj" level="WARN" />
+    <logger name="com.hierynomus.sshj" level="WARN" />
+
+    <!--
+        Logger for capturing user events. We do not want to propagate these
+        log events to the root logger. These messages are only sent to the
+        user-log appender.
+    -->
+    <logger name="org.apache.nifi.web.security" level="INFO" additivity="false">
+        <appender-ref ref="USER_FILE"/>
+    </logger>
+    <logger name="org.apache.nifi.web.api.config" level="INFO" additivity="false">
+        <appender-ref ref="USER_FILE"/>
+    </logger>
+    <logger name="org.apache.nifi.authorization" level="INFO" additivity="false">
+        <appender-ref ref="USER_FILE"/>
+    </logger>
+    <logger name="org.apache.nifi.cluster.authorization" level="INFO" additivity="false">
+        <appender-ref ref="USER_FILE"/>
+    </logger>
+    <logger name="org.apache.nifi.web.filter.RequestLogger" level="INFO" additivity="false">
+        <appender-ref ref="USER_FILE"/>
+    </logger>
+    <logger name="org.apache.nifi.web.api.AccessResource" level="INFO" additivity="false">
+        <appender-ref ref="USER_FILE"/>
+    </logger>
+
+
+    <!--
+        Logger for capturing Bootstrap logs and NiFi's standard error and standard out.
+    -->
+    <logger name="org.apache.nifi.bootstrap" level="INFO" additivity="false">
+        <appender-ref ref="BOOTSTRAP_FILE" />
+    </logger>
+    <logger name="org.apache.nifi.bootstrap.Command" level="INFO" additivity="false">
+        <appender-ref ref="CONSOLE" />
+        <appender-ref ref="BOOTSTRAP_FILE" />
+    </logger>
+
+    <!-- Everything written to NiFi's Standard Out will be logged with the logger org.apache.nifi.StdOut at INFO level -->
+    <logger name="org.apache.nifi.StdOut" level="INFO" additivity="false">
+        <appender-ref ref="BOOTSTRAP_FILE" />
+    </logger>
+
+    <!-- Everything written to NiFi's Standard Error will be logged with the logger org.apache.nifi.StdErr at ERROR level -->
+    <logger name="org.apache.nifi.StdErr" level="ERROR" additivity="false">
+        <appender-ref ref="BOOTSTRAP_FILE" />
+    </logger>
+
+
+    <root level="INFO">
+        <appender-ref ref="APP_FILE"/>
+    </root>
+
+</configuration>
\ No newline at end of file
diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/resources/conf/clustered/node2/nifi.properties b/nifi-system-tests/nifi-system-test-suite/src/test/resources/conf/clustered/node2/nifi.properties
new file mode 100644
index 0000000..67c1046
--- /dev/null
+++ b/nifi-system-tests/nifi-system-test-suite/src/test/resources/conf/clustered/node2/nifi.properties
@@ -0,0 +1,256 @@
+# 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.
+
+# Core Properties #
+nifi.flow.configuration.file=./conf/flow.xml.gz
+nifi.flow.configuration.archive.enabled=true
+nifi.flow.configuration.archive.dir=./conf/archive/
+nifi.flow.configuration.archive.max.time=30 days
+nifi.flow.configuration.archive.max.storage=500 MB
+nifi.flow.configuration.archive.max.count=
+nifi.flowcontroller.autoResumeState=true
+nifi.flowcontroller.graceful.shutdown.period=10 sec
+nifi.flowservice.writedelay.interval=500 ms
+nifi.administrative.yield.duration=100 millis
+# If a component has no work to do (is "bored"), how long should we wait before checking again for work?
+nifi.bored.yield.duration=10 millis
+nifi.queue.backpressure.count=10000
+nifi.queue.backpressure.size=1 GB
+
+nifi.authorizer.configuration.file=./conf/authorizers.xml
+nifi.login.identity.provider.configuration.file=./conf/login-identity-providers.xml
+nifi.templates.directory=./conf/templates
+nifi.ui.banner.text=
+nifi.ui.autorefresh.interval=30 sec
+nifi.nar.library.directory=./lib
+nifi.nar.library.autoload.directory=./extensions
+nifi.nar.working.directory=./work/nar/
+nifi.documentation.working.directory=./work/docs/components
+
+####################
+# State Management #
+####################
+nifi.state.management.configuration.file=./conf/state-management.xml
+# The ID of the local state provider
+nifi.state.management.provider.local=local-provider
+# The ID of the cluster-wide state provider. This will be ignored if NiFi is not clustered but must be populated if running in a cluster.
+nifi.state.management.provider.cluster=zk-provider
+# Specifies whether or not this instance of NiFi should run an embedded ZooKeeper server
+nifi.state.management.embedded.zookeeper.start=false
+# Properties file that provides the ZooKeeper properties to use if <nifi.state.management.embedded.zookeeper.start> is set to true
+nifi.state.management.embedded.zookeeper.properties=
+
+
+# H2 Settings
+nifi.database.directory=./database_repository
+nifi.h2.url.append=;LOCK_TIMEOUT=25000;WRITE_DELAY=0;AUTO_SERVER=FALSE
+
+# FlowFile Repository
+nifi.flowfile.repository.implementation=org.apache.nifi.controller.repository.WriteAheadFlowFileRepository
+nifi.flowfile.repository.wal.implementation=org.apache.nifi.wali.SequentialAccessWriteAheadLog
+nifi.flowfile.repository.directory=./flowfile_repository
+nifi.flowfile.repository.partitions=256
+nifi.flowfile.repository.checkpoint.interval=5 mins
+nifi.flowfile.repository.always.sync=false
+
+nifi.swap.manager.implementation=org.apache.nifi.controller.FileSystemSwapManager
+nifi.queue.swap.threshold=20000
+nifi.swap.in.period=5 sec
+nifi.swap.in.threads=1
+nifi.swap.out.period=5 sec
+nifi.swap.out.threads=4
+
+# Content Repository
+nifi.content.repository.implementation=org.apache.nifi.controller.repository.FileSystemRepository
+nifi.content.claim.max.appendable.size=1 MB
+nifi.content.claim.max.flow.files=100
+nifi.content.repository.directory.default=./content_repository
+nifi.content.repository.archive.max.retention.period=12 hours
+nifi.content.repository.archive.max.usage.percentage=50%
+nifi.content.repository.archive.enabled=true
+nifi.content.repository.always.sync=false
+nifi.content.viewer.url=../nifi-content-viewer/
+
+# Provenance Repository Properties
+nifi.provenance.repository.implementation=org.apache.nifi.provenance.WriteAheadProvenanceRepository
+nifi.provenance.repository.debug.frequency=1_000_000
+nifi.provenance.repository.encryption.key.provider.implementation=
+nifi.provenance.repository.encryption.key.provider.location=
+nifi.provenance.repository.encryption.key.id=
+nifi.provenance.repository.encryption.key=
+
+# Persistent Provenance Repository Properties
+nifi.provenance.repository.directory.default=./provenance_repository
+nifi.provenance.repository.max.storage.time=24 hours
+nifi.provenance.repository.max.storage.size=1 GB
+nifi.provenance.repository.rollover.time=30 secs
+nifi.provenance.repository.rollover.size=100 MB
+nifi.provenance.repository.query.threads=2
+nifi.provenance.repository.index.threads=2
+nifi.provenance.repository.compress.on.rollover=true
+nifi.provenance.repository.always.sync=false
+# Comma-separated list of fields. Fields that are not indexed will not be searchable. Valid fields are:
+# EventType, FlowFileUUID, Filename, TransitURI, ProcessorID, AlternateIdentifierURI, Relationship, Details
+nifi.provenance.repository.indexed.fields=EventType, FlowFileUUID, Filename, ProcessorID, Relationship
+# FlowFile Attributes that should be indexed and made searchable.  Some examples to consider are filename, uuid, mime.type
+nifi.provenance.repository.indexed.attributes=
+# Large values for the shard size will result in more Java heap usage when searching the Provenance Repository
+# but should provide better performance
+nifi.provenance.repository.index.shard.size=500 MB
+# Indicates the maximum length that a FlowFile attribute can be when retrieving a Provenance Event from
+# the repository. If the length of any attribute exceeds this value, it will be truncated when the event is retrieved.
+nifi.provenance.repository.max.attribute.length=65536
+nifi.provenance.repository.concurrent.merge.threads=2
+
+
+# Volatile Provenance Respository Properties
+nifi.provenance.repository.buffer.size=100000
+
+# Component Status Repository
+nifi.components.status.repository.implementation=org.apache.nifi.controller.status.history.VolatileComponentStatusRepository
+nifi.components.status.repository.buffer.size=1440
+nifi.components.status.snapshot.frequency=1 min
+
+# Site to Site properties
+nifi.remote.input.host=
+nifi.remote.input.secure=false
+nifi.remote.input.socket.port=7782
+nifi.remote.input.http.enabled=true
+nifi.remote.input.http.transaction.ttl=30 sec
+nifi.remote.contents.cache.expiration=30 secs
+
+# web properties #
+nifi.web.war.directory=./lib
+nifi.web.http.host=
+nifi.web.http.port=5672
+nifi.web.http.network.interface.default=
+nifi.web.https.host=
+nifi.web.https.port=
+nifi.web.https.network.interface.default=
+nifi.web.jetty.working.directory=./work/jetty
+nifi.web.jetty.threads=200
+nifi.web.max.header.size=16 KB
+nifi.web.proxy.context.path=
+nifi.web.proxy.host=
+
+# security properties #
+nifi.sensitive.props.key=
+nifi.sensitive.props.key.protected=
+nifi.sensitive.props.algorithm=PBEWITHMD5AND256BITAES-CBC-OPENSSL
+nifi.sensitive.props.provider=BC
+nifi.sensitive.props.additional.keys=
+
+nifi.security.keystore=certs/keystore.jks
+nifi.security.keystoreType=JKS
+nifi.security.keystorePasswd=passwordpassword
+nifi.security.keyPasswd=
+nifi.security.truststore=certs/truststore.jks
+nifi.security.truststoreType=JKS
+nifi.security.truststorePasswd=passwordpassword
+nifi.security.user.authorizer=managed-authorizer
+nifi.security.user.login.identity.provider=
+nifi.security.ocsp.responder.url=
+nifi.security.ocsp.responder.certificate=
+
+# OpenId Connect SSO Properties #
+nifi.security.user.oidc.discovery.url=
+nifi.security.user.oidc.connect.timeout=5 secs
+nifi.security.user.oidc.read.timeout=5 secs
+nifi.security.user.oidc.client.id=
+nifi.security.user.oidc.client.secret=
+nifi.security.user.oidc.preferred.jwsalgorithm=
+
+# Apache Knox SSO Properties #
+nifi.security.user.knox.url=
+nifi.security.user.knox.publicKey=
+nifi.security.user.knox.cookieName=hadoop-jwt
+nifi.security.user.knox.audiences=
+
+# Identity Mapping Properties #
+# These properties allow normalizing user identities such that identities coming from different identity providers
+# (certificates, LDAP, Kerberos) can be treated the same internally in NiFi. The following example demonstrates normalizing
+# DNs from certificates and principals from Kerberos into a common identity string:
+#
+# nifi.security.identity.mapping.pattern.dn=^CN=(.*?), OU=(.*?), O=(.*?), L=(.*?), ST=(.*?), C=(.*?)$
+# nifi.security.identity.mapping.value.dn=$1@$2
+# nifi.security.identity.mapping.transform.dn=NONE
+# nifi.security.identity.mapping.pattern.kerb=^(.*?)/instance@(.*?)$
+# nifi.security.identity.mapping.value.kerb=$1@$2
+# nifi.security.identity.mapping.transform.kerb=UPPER
+
+# Group Mapping Properties #
+# These properties allow normalizing group names coming from external sources like LDAP. The following example
+# lowercases any group name.
+#
+# nifi.security.group.mapping.pattern.anygroup=^(.*)$
+# nifi.security.group.mapping.value.anygroup=$1
+# nifi.security.group.mapping.transform.anygroup=LOWER
+
+# cluster common properties (all nodes must have same values) #
+nifi.cluster.protocol.heartbeat.interval=5 sec
+nifi.cluster.protocol.is.secure=false
+
+# cluster node properties (only configure for cluster nodes) #
+nifi.cluster.is.node=true
+nifi.cluster.node.address=
+nifi.cluster.node.protocol.port=48102
+nifi.cluster.node.protocol.threads=10
+nifi.cluster.node.protocol.max.threads=50
+nifi.cluster.node.event.history.size=25
+nifi.cluster.node.connection.timeout=5 sec
+nifi.cluster.node.read.timeout=5 sec
+nifi.cluster.node.max.concurrent.requests=100
+nifi.cluster.firewall.file=
+nifi.cluster.flow.election.max.wait.time=1 secs
+nifi.cluster.flow.election.max.candidates=1
+
+# cluster load balancing properties #
+nifi.cluster.load.balance.host=
+nifi.cluster.load.balance.port=6342
+nifi.cluster.load.balance.connections.per.node=4
+nifi.cluster.load.balance.max.thread.count=8
+nifi.cluster.load.balance.comms.timeout=30 sec
+
+# zookeeper properties, used for cluster management #
+nifi.zookeeper.connect.string=localhost:21818
+nifi.zookeeper.connect.timeout=3 secs
+nifi.zookeeper.session.timeout=3 secs
+nifi.zookeeper.root.node=/nifi
+
+# Zookeeper properties for the authentication scheme used when creating acls on znodes used for cluster management
+# Values supported for nifi.zookeeper.auth.type are "default", which will apply world/anyone rights on znodes
+# and "sasl" which will give rights to the sasl/kerberos identity used to authenticate the nifi node
+# The identity is determined using the value in nifi.kerberos.service.principal and the removeHostFromPrincipal
+# and removeRealmFromPrincipal values (which should align with the kerberos.removeHostFromPrincipal and kerberos.removeRealmFromPrincipal
+# values configured on the zookeeper server).
+nifi.zookeeper.auth.type=
+nifi.zookeeper.kerberos.removeHostFromPrincipal=
+nifi.zookeeper.kerberos.removeRealmFromPrincipal=
+
+# kerberos #
+nifi.kerberos.krb5.file=
+
+# kerberos service principal #
+nifi.kerberos.service.principal=
+nifi.kerberos.service.keytab.location=
+
+# kerberos spnego principal #
+nifi.kerberos.spnego.principal=
+nifi.kerberos.spnego.keytab.location=
+nifi.kerberos.spnego.authentication.expiration=12 hours
+
+# external properties files for variable registry
+# supports a comma delimited list of file locations
+nifi.variable.registry.properties=
\ No newline at end of file
diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/resources/conf/clustered/node2/state-management.xml b/nifi-system-tests/nifi-system-test-suite/src/test/resources/conf/clustered/node2/state-management.xml
new file mode 100644
index 0000000..3dacd13
--- /dev/null
+++ b/nifi-system-tests/nifi-system-test-suite/src/test/resources/conf/clustered/node2/state-management.xml
@@ -0,0 +1,32 @@
+<!--
+  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.
+-->
+<stateManagement>
+    <local-provider>
+        <id>local-provider</id>
+        <class>org.apache.nifi.controller.state.providers.local.WriteAheadLocalStateProvider</class>
+        <property name="Directory">./state/local</property>
+        <property name="Always Sync">false</property>
+        <property name="Partitions">16</property>
+        <property name="Checkpoint Interval">2 mins</property>
+    </local-provider>
+    <cluster-provider>
+        <id>zk-provider</id>
+        <class>org.apache.nifi.controller.state.providers.zookeeper.ZooKeeperStateProvider</class>
+        <property name="Connect String">localhost:21818</property>
+        <property name="Root Node">/nifi-system-test</property>
+        <property name="Session Timeout">30 seconds</property>
+        <property name="Access Control">Open</property>
+    </cluster-provider>
+</stateManagement>
\ No newline at end of file
diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/resources/conf/default/bootstrap.conf b/nifi-system-tests/nifi-system-test-suite/src/test/resources/conf/default/bootstrap.conf
new file mode 100644
index 0000000..3dc2c29
--- /dev/null
+++ b/nifi-system-tests/nifi-system-test-suite/src/test/resources/conf/default/bootstrap.conf
@@ -0,0 +1,33 @@
+# 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.
+
+# Configure where NiFi's lib and conf directories live
+lib.dir=../nifi-lib-assembly/lib
+conf.dir=./conf
+working.dir=./target/standalone-instance
+#java=/Library/Java/JavaVirtualMachines/jdk1.8.0_60.jdk/Contents/Home/bin/java
+
+# How long to wait after telling NiFi to shutdown before explicitly killing the Process
+graceful.shutdown.seconds=20
+
+# JVM memory settings
+java.arg.2= -Xms128m
+java.arg.3=-Xmx128m
+
+java.arg.14=-Djava.awt.headless=true
+
+#java.arg.debug=-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=8002
+
+nifi.bootstrap.sensitive.key=
\ No newline at end of file
diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/resources/conf/default/logback.xml b/nifi-system-tests/nifi-system-test-suite/src/test/resources/conf/default/logback.xml
new file mode 100644
index 0000000..73c5cdc
--- /dev/null
+++ b/nifi-system-tests/nifi-system-test-suite/src/test/resources/conf/default/logback.xml
@@ -0,0 +1,178 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+
+<configuration scan="true" scanPeriod="30 seconds">
+    <contextListener class="ch.qos.logback.classic.jul.LevelChangePropagator">
+        <resetJUL>true</resetJUL>
+    </contextListener>
+
+    <appender name="APP_FILE" class="ch.qos.logback.core.rolling.RollingFileAppender">
+        <file>${org.apache.nifi.bootstrap.config.log.dir}/nifi-app.log</file>
+        <rollingPolicy class="ch.qos.logback.core.rolling.SizeAndTimeBasedRollingPolicy">
+            <!--
+              For daily rollover, use 'app_%d.log'.
+              For hourly rollover, use 'app_%d{yyyy-MM-dd_HH}.log'.
+              To GZIP rolled files, replace '.log' with '.log.gz'.
+              To ZIP rolled files, replace '.log' with '.log.zip'.
+            -->
+            <fileNamePattern>${org.apache.nifi.bootstrap.config.log.dir}/nifi-app_%d{yyyy-MM-dd_HH}.%i.log</fileNamePattern>
+            <maxFileSize>100MB</maxFileSize>
+            <!-- keep 30 log files worth of history -->
+            <maxHistory>30</maxHistory>
+        </rollingPolicy>
+        <immediateFlush>true</immediateFlush>
+        <encoder class="ch.qos.logback.classic.encoder.PatternLayoutEncoder">
+            <pattern>%date %level [%thread] %logger{40} %msg%n</pattern>
+        </encoder>
+    </appender>
+
+    <appender name="USER_FILE" class="ch.qos.logback.core.rolling.RollingFileAppender">
+        <file>${org.apache.nifi.bootstrap.config.log.dir}/nifi-user.log</file>
+        <rollingPolicy class="ch.qos.logback.core.rolling.TimeBasedRollingPolicy">
+            <!--
+              For daily rollover, use 'user_%d.log'.
+              For hourly rollover, use 'user_%d{yyyy-MM-dd_HH}.log'.
+              To GZIP rolled files, replace '.log' with '.log.gz'.
+              To ZIP rolled files, replace '.log' with '.log.zip'.
+            -->
+            <fileNamePattern>${org.apache.nifi.bootstrap.config.log.dir}/nifi-user_%d.log</fileNamePattern>
+            <!-- keep 30 log files worth of history -->
+            <maxHistory>30</maxHistory>
+        </rollingPolicy>
+        <encoder class="ch.qos.logback.classic.encoder.PatternLayoutEncoder">
+            <pattern>%date %level [%thread] %logger{40} %msg%n</pattern>
+        </encoder>
+    </appender>
+
+    <appender name="BOOTSTRAP_FILE" class="ch.qos.logback.core.rolling.RollingFileAppender">
+        <file>${org.apache.nifi.bootstrap.config.log.dir}/nifi-bootstrap.log</file>
+        <rollingPolicy class="ch.qos.logback.core.rolling.TimeBasedRollingPolicy">
+            <!--
+              For daily rollover, use 'user_%d.log'.
+              For hourly rollover, use 'user_%d{yyyy-MM-dd_HH}.log'.
+              To GZIP rolled files, replace '.log' with '.log.gz'.
+              To ZIP rolled files, replace '.log' with '.log.zip'.
+            -->
+            <fileNamePattern>${org.apache.nifi.bootstrap.config.log.dir}/nifi-bootstrap_%d.log</fileNamePattern>
+            <!-- keep 5 log files worth of history -->
+            <maxHistory>5</maxHistory>
+        </rollingPolicy>
+        <encoder class="ch.qos.logback.classic.encoder.PatternLayoutEncoder">
+            <pattern>%date %level [%thread] %logger{40} %msg%n</pattern>
+        </encoder>
+    </appender>
+
+    <appender name="CONSOLE" class="ch.qos.logback.core.ConsoleAppender">
+        <encoder class="ch.qos.logback.classic.encoder.PatternLayoutEncoder">
+            <pattern>%date %level [%thread] %logger{40} %msg%n</pattern>
+        </encoder>
+    </appender>
+
+    <!-- valid logging levels: TRACE, DEBUG, INFO, WARN, ERROR -->
+    <logger name="org.apache.nifi.web.util.LocalComponentLifecycle" level="DEBUG" />
+    <logger name="org.apache.nifi" level="INFO"/>
+    <logger name="org.apache.nifi.processors" level="WARN"/>
+    <logger name="org.apache.nifi.processors.standard.LogAttribute" level="INFO"/>
+    <logger name="org.apache.nifi.processors.standard.LogMessage" level="INFO"/>
+    <logger name="org.apache.nifi.controller.repository.StandardProcessSession" level="WARN" />
+
+
+    <logger name="org.apache.zookeeper.ClientCnxn" level="ERROR" />
+    <logger name="org.apache.zookeeper.server.NIOServerCnxn" level="ERROR" />
+    <logger name="org.apache.zookeeper.server.NIOServerCnxnFactory" level="ERROR" />
+    <logger name="org.apache.zookeeper.server.quorum" level="ERROR" />
+    <logger name="org.apache.zookeeper.ZooKeeper" level="ERROR" />
+    <logger name="org.apache.zookeeper.server.PrepRequestProcessor" level="ERROR" />
+
+    <logger name="org.apache.calcite.runtime.CalciteException" level="OFF" />
+
+    <logger name="org.apache.curator.framework.recipes.leader.LeaderSelector" level="OFF" />
+    <logger name="org.apache.curator.ConnectionState" level="OFF" />
+
+    <!-- Logger for managing logging statements for nifi clusters. -->
+    <logger name="org.apache.nifi.cluster" level="INFO"/>
+
+    <!-- Logger for logging HTTP requests received by the web server. -->
+    <logger name="org.apache.nifi.server.JettyServer" level="INFO"/>
+
+    <!-- Logger for managing logging statements for jetty -->
+    <logger name="org.eclipse.jetty" level="INFO"/>
+
+    <!-- Suppress non-error messages due to excessive logging by class or library -->
+    <logger name="org.springframework" level="ERROR"/>
+
+    <!-- Suppress non-error messages due to known warning about redundant path annotation (NIFI-574) -->
+    <logger name="org.glassfish.jersey.internal.Errors" level="ERROR"/>
+
+    <!-- Suppress non-error messages due to Jetty AnnotationParser emitting a large amount of WARNS. Issue described in NIFI-5479. -->
+    <logger name="org.eclipse.jetty.annotations.AnnotationParser" level="ERROR"/>
+
+    <!-- Suppress non-error messages from SSHJ which was emitting large amounts of INFO logs by default -->
+    <logger name="net.schmizz.sshj" level="WARN" />
+    <logger name="com.hierynomus.sshj" level="WARN" />
+
+    <!--
+        Logger for capturing user events. We do not want to propagate these
+        log events to the root logger. These messages are only sent to the
+        user-log appender.
+    -->
+    <logger name="org.apache.nifi.web.security" level="INFO" additivity="false">
+        <appender-ref ref="USER_FILE"/>
+    </logger>
+    <logger name="org.apache.nifi.web.api.config" level="INFO" additivity="false">
+        <appender-ref ref="USER_FILE"/>
+    </logger>
+    <logger name="org.apache.nifi.authorization" level="INFO" additivity="false">
+        <appender-ref ref="USER_FILE"/>
+    </logger>
+    <logger name="org.apache.nifi.cluster.authorization" level="INFO" additivity="false">
+        <appender-ref ref="USER_FILE"/>
+    </logger>
+    <logger name="org.apache.nifi.web.filter.RequestLogger" level="INFO" additivity="false">
+        <appender-ref ref="USER_FILE"/>
+    </logger>
+    <logger name="org.apache.nifi.web.api.AccessResource" level="INFO" additivity="false">
+        <appender-ref ref="USER_FILE"/>
+    </logger>
+
+
+    <!--
+        Logger for capturing Bootstrap logs and NiFi's standard error and standard out.
+    -->
+    <logger name="org.apache.nifi.bootstrap" level="INFO" additivity="false">
+        <appender-ref ref="BOOTSTRAP_FILE" />
+    </logger>
+    <logger name="org.apache.nifi.bootstrap.Command" level="INFO" additivity="false">
+        <appender-ref ref="CONSOLE" />
+        <appender-ref ref="BOOTSTRAP_FILE" />
+    </logger>
+
+    <!-- Everything written to NiFi's Standard Out will be logged with the logger org.apache.nifi.StdOut at INFO level -->
+    <logger name="org.apache.nifi.StdOut" level="INFO" additivity="false">
+        <appender-ref ref="BOOTSTRAP_FILE" />
+    </logger>
+
+    <!-- Everything written to NiFi's Standard Error will be logged with the logger org.apache.nifi.StdErr at ERROR level -->
+    <logger name="org.apache.nifi.StdErr" level="ERROR" additivity="false">
+        <appender-ref ref="BOOTSTRAP_FILE" />
+    </logger>
+
+
+    <root level="INFO">
+        <appender-ref ref="APP_FILE"/>
+    </root>
+
+</configuration>
\ No newline at end of file
diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/resources/conf/default/nifi.properties b/nifi-system-tests/nifi-system-test-suite/src/test/resources/conf/default/nifi.properties
new file mode 100644
index 0000000..0b4aa9d
--- /dev/null
+++ b/nifi-system-tests/nifi-system-test-suite/src/test/resources/conf/default/nifi.properties
@@ -0,0 +1,256 @@
+# 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.
+
+# Core Properties #
+nifi.flow.configuration.file=./conf/flow.xml.gz
+nifi.flow.configuration.archive.enabled=true
+nifi.flow.configuration.archive.dir=./conf/archive/
+nifi.flow.configuration.archive.max.time=30 days
+nifi.flow.configuration.archive.max.storage=500 MB
+nifi.flow.configuration.archive.max.count=
+nifi.flowcontroller.autoResumeState=true
+nifi.flowcontroller.graceful.shutdown.period=10 sec
+nifi.flowservice.writedelay.interval=500 ms
+nifi.administrative.yield.duration=100 millis
+# If a component has no work to do (is "bored"), how long should we wait before checking again for work?
+nifi.bored.yield.duration=10 millis
+nifi.queue.backpressure.count=10000
+nifi.queue.backpressure.size=1 GB
+
+nifi.authorizer.configuration.file=./conf/authorizers.xml
+nifi.login.identity.provider.configuration.file=./conf/login-identity-providers.xml
+nifi.templates.directory=./conf/templates
+nifi.ui.banner.text=
+nifi.ui.autorefresh.interval=30 sec
+nifi.nar.library.directory=./lib
+nifi.nar.library.autoload.directory=./extensions
+nifi.nar.working.directory=./work/nar/
+nifi.documentation.working.directory=./work/docs/components
+
+####################
+# State Management #
+####################
+nifi.state.management.configuration.file=conf/state-management.xml
+# The ID of the local state provider
+nifi.state.management.provider.local=local-provider
+# The ID of the cluster-wide state provider. This will be ignored if NiFi is not clustered but must be populated if running in a cluster.
+nifi.state.management.provider.cluster=zk-provider
+# Specifies whether or not this instance of NiFi should run an embedded ZooKeeper server
+nifi.state.management.embedded.zookeeper.start=false
+# Properties file that provides the ZooKeeper properties to use if <nifi.state.management.embedded.zookeeper.start> is set to true
+nifi.state.management.embedded.zookeeper.properties=./conf/zookeeper.properties
+
+
+# H2 Settings
+nifi.database.directory=./database_repository
+nifi.h2.url.append=;LOCK_TIMEOUT=25000;WRITE_DELAY=0;AUTO_SERVER=FALSE
+
+# FlowFile Repository
+nifi.flowfile.repository.implementation=org.apache.nifi.controller.repository.WriteAheadFlowFileRepository
+nifi.flowfile.repository.wal.implementation=org.apache.nifi.wali.SequentialAccessWriteAheadLog
+nifi.flowfile.repository.directory=./flowfile_repository
+nifi.flowfile.repository.partitions=256
+nifi.flowfile.repository.checkpoint.interval=5 mins
+nifi.flowfile.repository.always.sync=false
+
+nifi.swap.manager.implementation=org.apache.nifi.controller.FileSystemSwapManager
+nifi.queue.swap.threshold=20000
+nifi.swap.in.period=5 sec
+nifi.swap.in.threads=1
+nifi.swap.out.period=5 sec
+nifi.swap.out.threads=4
+
+# Content Repository
+nifi.content.repository.implementation=org.apache.nifi.controller.repository.FileSystemRepository
+nifi.content.claim.max.appendable.size=1 MB
+nifi.content.claim.max.flow.files=100
+nifi.content.repository.directory.default=./content_repository
+nifi.content.repository.archive.max.retention.period=12 hours
+nifi.content.repository.archive.max.usage.percentage=50%
+nifi.content.repository.archive.enabled=true
+nifi.content.repository.always.sync=false
+nifi.content.viewer.url=../nifi-content-viewer/
+
+# Provenance Repository Properties
+nifi.provenance.repository.implementation=org.apache.nifi.provenance.WriteAheadProvenanceRepository
+nifi.provenance.repository.debug.frequency=1_000_000
+nifi.provenance.repository.encryption.key.provider.implementation=
+nifi.provenance.repository.encryption.key.provider.location=
+nifi.provenance.repository.encryption.key.id=
+nifi.provenance.repository.encryption.key=
+
+# Persistent Provenance Repository Properties
+nifi.provenance.repository.directory.default=./provenance_repository
+nifi.provenance.repository.max.storage.time=24 hours
+nifi.provenance.repository.max.storage.size=1 GB
+nifi.provenance.repository.rollover.time=30 secs
+nifi.provenance.repository.rollover.size=100 MB
+nifi.provenance.repository.query.threads=2
+nifi.provenance.repository.index.threads=2
+nifi.provenance.repository.compress.on.rollover=true
+nifi.provenance.repository.always.sync=false
+# Comma-separated list of fields. Fields that are not indexed will not be searchable. Valid fields are:
+# EventType, FlowFileUUID, Filename, TransitURI, ProcessorID, AlternateIdentifierURI, Relationship, Details
+nifi.provenance.repository.indexed.fields=EventType, FlowFileUUID, Filename, ProcessorID, Relationship
+# FlowFile Attributes that should be indexed and made searchable.  Some examples to consider are filename, uuid, mime.type
+nifi.provenance.repository.indexed.attributes=
+# Large values for the shard size will result in more Java heap usage when searching the Provenance Repository
+# but should provide better performance
+nifi.provenance.repository.index.shard.size=500 MB
+# Indicates the maximum length that a FlowFile attribute can be when retrieving a Provenance Event from
+# the repository. If the length of any attribute exceeds this value, it will be truncated when the event is retrieved.
+nifi.provenance.repository.max.attribute.length=65536
+nifi.provenance.repository.concurrent.merge.threads=2
+
+
+# Volatile Provenance Respository Properties
+nifi.provenance.repository.buffer.size=100000
+
+# Component Status Repository
+nifi.components.status.repository.implementation=org.apache.nifi.controller.status.history.VolatileComponentStatusRepository
+nifi.components.status.repository.buffer.size=1440
+nifi.components.status.snapshot.frequency=1 min
+
+# Site to Site properties
+nifi.remote.input.host=
+nifi.remote.input.secure=false
+nifi.remote.input.socket.port=7781
+nifi.remote.input.http.enabled=true
+nifi.remote.input.http.transaction.ttl=30 sec
+nifi.remote.contents.cache.expiration=30 secs
+
+# web properties #
+nifi.web.war.directory=./lib
+nifi.web.http.host=
+nifi.web.http.port=5671
+nifi.web.http.network.interface.default=
+nifi.web.https.host=
+nifi.web.https.port=
+nifi.web.https.network.interface.default=
+nifi.web.jetty.working.directory=./work/jetty
+nifi.web.jetty.threads=200
+nifi.web.max.header.size=16 KB
+nifi.web.proxy.context.path=
+nifi.web.proxy.host=
+
+# security properties #
+nifi.sensitive.props.key=
+nifi.sensitive.props.key.protected=
+nifi.sensitive.props.algorithm=PBEWITHMD5AND256BITAES-CBC-OPENSSL
+nifi.sensitive.props.provider=BC
+nifi.sensitive.props.additional.keys=
+
+nifi.security.keystore=certs/keystore.jks
+nifi.security.keystoreType=JKS
+nifi.security.keystorePasswd=passwordpassword
+nifi.security.keyPasswd=
+nifi.security.truststore=certs/truststore.jks
+nifi.security.truststoreType=JKS
+nifi.security.truststorePasswd=passwordpassword
+nifi.security.user.authorizer=managed-authorizer
+nifi.security.user.login.identity.provider=
+nifi.security.ocsp.responder.url=
+nifi.security.ocsp.responder.certificate=
+
+# OpenId Connect SSO Properties #
+nifi.security.user.oidc.discovery.url=
+nifi.security.user.oidc.connect.timeout=5 secs
+nifi.security.user.oidc.read.timeout=5 secs
+nifi.security.user.oidc.client.id=
+nifi.security.user.oidc.client.secret=
+nifi.security.user.oidc.preferred.jwsalgorithm=
+
+# Apache Knox SSO Properties #
+nifi.security.user.knox.url=
+nifi.security.user.knox.publicKey=
+nifi.security.user.knox.cookieName=hadoop-jwt
+nifi.security.user.knox.audiences=
+
+# Identity Mapping Properties #
+# These properties allow normalizing user identities such that identities coming from different identity providers
+# (certificates, LDAP, Kerberos) can be treated the same internally in NiFi. The following example demonstrates normalizing
+# DNs from certificates and principals from Kerberos into a common identity string:
+#
+# nifi.security.identity.mapping.pattern.dn=^CN=(.*?), OU=(.*?), O=(.*?), L=(.*?), ST=(.*?), C=(.*?)$
+# nifi.security.identity.mapping.value.dn=$1@$2
+# nifi.security.identity.mapping.transform.dn=NONE
+# nifi.security.identity.mapping.pattern.kerb=^(.*?)/instance@(.*?)$
+# nifi.security.identity.mapping.value.kerb=$1@$2
+# nifi.security.identity.mapping.transform.kerb=UPPER
+
+# Group Mapping Properties #
+# These properties allow normalizing group names coming from external sources like LDAP. The following example
+# lowercases any group name.
+#
+# nifi.security.group.mapping.pattern.anygroup=^(.*)$
+# nifi.security.group.mapping.value.anygroup=$1
+# nifi.security.group.mapping.transform.anygroup=LOWER
+
+# cluster common properties (all nodes must have same values) #
+nifi.cluster.protocol.heartbeat.interval=5 sec
+nifi.cluster.protocol.is.secure=false
+
+# cluster node properties (only configure for cluster nodes) #
+nifi.cluster.is.node=false
+nifi.cluster.node.address=
+nifi.cluster.node.protocol.port=
+nifi.cluster.node.protocol.threads=10
+nifi.cluster.node.protocol.max.threads=50
+nifi.cluster.node.event.history.size=25
+nifi.cluster.node.connection.timeout=5 sec
+nifi.cluster.node.read.timeout=5 sec
+nifi.cluster.node.max.concurrent.requests=100
+nifi.cluster.firewall.file=
+nifi.cluster.flow.election.max.wait.time=5 mins
+nifi.cluster.flow.election.max.candidates=
+
+# cluster load balancing properties #
+nifi.cluster.load.balance.host=
+nifi.cluster.load.balance.port=6342
+nifi.cluster.load.balance.connections.per.node=4
+nifi.cluster.load.balance.max.thread.count=8
+nifi.cluster.load.balance.comms.timeout=30 sec
+
+# zookeeper properties, used for cluster management #
+nifi.zookeeper.connect.string=
+nifi.zookeeper.connect.timeout=3 secs
+nifi.zookeeper.session.timeout=3 secs
+nifi.zookeeper.root.node=/nifi
+
+# Zookeeper properties for the authentication scheme used when creating acls on znodes used for cluster management
+# Values supported for nifi.zookeeper.auth.type are "default", which will apply world/anyone rights on znodes
+# and "sasl" which will give rights to the sasl/kerberos identity used to authenticate the nifi node
+# The identity is determined using the value in nifi.kerberos.service.principal and the removeHostFromPrincipal
+# and removeRealmFromPrincipal values (which should align with the kerberos.removeHostFromPrincipal and kerberos.removeRealmFromPrincipal
+# values configured on the zookeeper server).
+nifi.zookeeper.auth.type=
+nifi.zookeeper.kerberos.removeHostFromPrincipal=
+nifi.zookeeper.kerberos.removeRealmFromPrincipal=
+
+# kerberos #
+nifi.kerberos.krb5.file=
+
+# kerberos service principal #
+nifi.kerberos.service.principal=
+nifi.kerberos.service.keytab.location=
+
+# kerberos spnego principal #
+nifi.kerberos.spnego.principal=
+nifi.kerberos.spnego.keytab.location=
+nifi.kerberos.spnego.authentication.expiration=12 hours
+
+# external properties files for variable registry
+# supports a comma delimited list of file locations
+nifi.variable.registry.properties=
\ No newline at end of file
diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/resources/conf/default/state-management.xml b/nifi-system-tests/nifi-system-test-suite/src/test/resources/conf/default/state-management.xml
new file mode 100644
index 0000000..65f6d9c
--- /dev/null
+++ b/nifi-system-tests/nifi-system-test-suite/src/test/resources/conf/default/state-management.xml
@@ -0,0 +1,32 @@
+<!--
+  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.
+-->
+<stateManagement>
+    <local-provider>
+        <id>local-provider</id>
+        <class>org.apache.nifi.controller.state.providers.local.WriteAheadLocalStateProvider</class>
+        <property name="Directory">./state/local</property>
+        <property name="Always Sync">false</property>
+        <property name="Partitions">16</property>
+        <property name="Checkpoint Interval">2 mins</property>
+    </local-provider>
+    <cluster-provider>
+        <id>zk-provider</id>
+        <class>org.apache.nifi.controller.state.providers.zookeeper.ZooKeeperStateProvider</class>
+        <property name="Connect String">localhost:62181</property>
+        <property name="Root Node">/nifi-integration-test</property>
+        <property name="Session Timeout">30 seconds</property>
+        <property name="Access Control">Open</property>
+    </cluster-provider>
+</stateManagement>
\ No newline at end of file
diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/resources/conf/default/zookeeper.properties b/nifi-system-tests/nifi-system-test-suite/src/test/resources/conf/default/zookeeper.properties
new file mode 100644
index 0000000..47b9290
--- /dev/null
+++ b/nifi-system-tests/nifi-system-test-suite/src/test/resources/conf/default/zookeeper.properties
@@ -0,0 +1,45 @@
+#
+#
+# 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.
+#
+#
+#
+
+clientPort=62181
+initLimit=10
+autopurge.purgeInterval=24
+syncLimit=5
+tickTime=2000
+dataDir=target/state/zookeeper
+autopurge.snapRetainCount=30
+
+#
+# Specifies the servers that are part of this zookeeper ensemble. For
+# every NiFi instance running an embedded zookeeper, there needs to be
+# a server entry below. For instance:
+#
+# server.1=nifi-node1-hostname:2888:3888
+# server.2=nifi-node2-hostname:2888:3888
+# server.3=nifi-node3-hostname:2888:3888
+#
+# The index of the server corresponds to the myid file that gets created
+# in the dataDir of each node running an embedded zookeeper. See the
+# administration guide for more details.
+#
+
+server.1=localhost:5777:6777
diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/resources/keystore.jks b/nifi-system-tests/nifi-system-test-suite/src/test/resources/keystore.jks
new file mode 100644
index 0000000..34a197f
Binary files /dev/null and b/nifi-system-tests/nifi-system-test-suite/src/test/resources/keystore.jks differ
diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/resources/truststore.jks b/nifi-system-tests/nifi-system-test-suite/src/test/resources/truststore.jks
new file mode 100644
index 0000000..4bc1b20
Binary files /dev/null and b/nifi-system-tests/nifi-system-test-suite/src/test/resources/truststore.jks differ
diff --git a/nifi-system-tests/pom.xml b/nifi-system-tests/pom.xml
new file mode 100644
index 0000000..dba2c3c
--- /dev/null
+++ b/nifi-system-tests/pom.xml
@@ -0,0 +1,33 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <artifactId>nifi</artifactId>
+        <groupId>org.apache.nifi</groupId>
+        <version>1.11.0-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>nifi-system-tests</artifactId>
+    <packaging>pom</packaging>
+
+    <modules>
+        <module>nifi-system-test-extensions-bundle</module>
+        <module>nifi-system-test-suite</module>
+    </modules>
+
+</project>
diff --git a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/NiFiClientFactory.java b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/NiFiClientFactory.java
index 07e4d41..499157c 100644
--- a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/NiFiClientFactory.java
+++ b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/NiFiClientFactory.java
@@ -20,14 +20,20 @@ import org.apache.commons.cli.MissingOptionException;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.registry.security.util.KeystoreType;
 import org.apache.nifi.toolkit.cli.api.ClientFactory;
+import org.apache.nifi.toolkit.cli.impl.client.nifi.ConnectionClient;
 import org.apache.nifi.toolkit.cli.impl.client.nifi.ControllerClient;
 import org.apache.nifi.toolkit.cli.impl.client.nifi.ControllerServicesClient;
+import org.apache.nifi.toolkit.cli.impl.client.nifi.CountersClient;
 import org.apache.nifi.toolkit.cli.impl.client.nifi.FlowClient;
+import org.apache.nifi.toolkit.cli.impl.client.nifi.InputPortClient;
 import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClient;
 import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientConfig;
+import org.apache.nifi.toolkit.cli.impl.client.nifi.OutputPortClient;
 import org.apache.nifi.toolkit.cli.impl.client.nifi.ParamContextClient;
 import org.apache.nifi.toolkit.cli.impl.client.nifi.PoliciesClient;
 import org.apache.nifi.toolkit.cli.impl.client.nifi.ProcessGroupClient;
+import org.apache.nifi.toolkit.cli.impl.client.nifi.ProcessorClient;
+import org.apache.nifi.toolkit.cli.impl.client.nifi.RemoteProcessGroupClient;
 import org.apache.nifi.toolkit.cli.impl.client.nifi.ReportingTasksClient;
 import org.apache.nifi.toolkit.cli.impl.client.nifi.TemplatesClient;
 import org.apache.nifi.toolkit.cli.impl.client.nifi.TenantsClient;
@@ -187,6 +193,21 @@ public class NiFiClientFactory implements ClientFactory<NiFiClient> {
         }
 
         @Override
+        public ProcessorClient getProcessorClient() {
+            return wrappedClient.getProcessorClientForProxiedEntities(proxiedEntity);
+        }
+
+        @Override
+        public ProcessorClient getProcessorClientForProxiedEntities(final String... proxiedEntity) {
+            return wrappedClient.getProcessorClientForProxiedEntities(proxiedEntity);
+        }
+
+        @Override
+        public ProcessorClient getProcessorClientForToken(final String token) {
+            return wrappedClient.getProcessorClientForToken(token);
+        }
+
+        @Override
         public VersionsClient getVersionsClient() {
             return wrappedClient.getVersionsClientForProxiedEntities(proxiedEntity);
         }
@@ -277,6 +298,81 @@ public class NiFiClientFactory implements ClientFactory<NiFiClient> {
         }
 
         @Override
+        public CountersClient getCountersClient() {
+            return wrappedClient.getCountersClientForProxiedEntities(proxiedEntity);
+        }
+
+        @Override
+        public CountersClient getCountersClientForProxiedEntities(final String... proxiedEntity) {
+            return wrappedClient.getCountersClientForProxiedEntities(proxiedEntity);
+        }
+
+        @Override
+        public CountersClient getCountersClientForToken(final String token) {
+            return wrappedClient.getCountersClientForToken(token);
+        }
+
+        @Override
+        public ConnectionClient getConnectionClient() {
+            return wrappedClient.getConnectionClientForProxiedEntities(proxiedEntity);
+        }
+
+        @Override
+        public ConnectionClient getConnectionClientForProxiedEntities(final String... proxiedEntity) {
+            return wrappedClient.getConnectionClientForProxiedEntities(proxiedEntity);
+        }
+
+        @Override
+        public ConnectionClient getConnectionClientForToken(final String token) {
+            return wrappedClient.getConnectionClientForToken(token);
+        }
+
+        @Override
+        public RemoteProcessGroupClient getRemoteProcessGroupClient() {
+            return wrappedClient.getRemoteProcessGroupClientForProxiedEntities(proxiedEntity);
+        }
+
+        @Override
+        public RemoteProcessGroupClient getRemoteProcessGroupClientForProxiedEntities(final String... proxiedEntity) {
+            return wrappedClient.getRemoteProcessGroupClientForProxiedEntities(proxiedEntity);
+        }
+
+        @Override
+        public RemoteProcessGroupClient getRemoteProcessGroupClientForToken(final String token) {
+            return wrappedClient.getRemoteProcessGroupClientForToken(token);
+        }
+
+        @Override
+        public InputPortClient getInputPortClient() {
+            return wrappedClient.getInputPortClientForProxiedEntities(proxiedEntity);
+        }
+
+        @Override
+        public InputPortClient getInputPortClientForProxiedEntities(final String... proxiedEntity) {
+            return wrappedClient.getInputPortClientForProxiedEntities(proxiedEntity);
+        }
+
+        @Override
+        public InputPortClient getInputPortClientForToken(final String token) {
+            return wrappedClient.getInputPortClientForToken(proxiedEntity);
+        }
+
+        @Override
+        public OutputPortClient getOutputPortClient() {
+            return wrappedClient.getOutputPortClientForProxiedEntities(proxiedEntity);
+        }
+
+        @Override
+        public OutputPortClient getOutputPortClientForProxiedEntities(final String... proxiedEntity) {
+            return wrappedClient.getOutputPortClientForProxiedEntities(proxiedEntity);
+        }
+
+        @Override
+        public OutputPortClient getOutputPortClientForToken(final String token) {
+            return wrappedClient.getOutputPortClientForToken(proxiedEntity);
+        }
+
+        @Override
         public void close() throws IOException {
             wrappedClient.close();
         }
diff --git a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/ConnectionClient.java b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/ConnectionClient.java
new file mode 100644
index 0000000..af5b5e9
--- /dev/null
+++ b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/ConnectionClient.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.toolkit.cli.impl.client.nifi;
+
+import org.apache.nifi.web.api.entity.ConnectionEntity;
+import org.apache.nifi.web.api.entity.DropRequestEntity;
+
+import java.io.IOException;
+
+public interface ConnectionClient {
+    ConnectionEntity getConnection(String id) throws NiFiClientException, IOException;
+
+    ConnectionEntity deleteConnection(ConnectionEntity connectionEntity) throws NiFiClientException, IOException;
+
+    ConnectionEntity deleteConnection(String id, String clientId, long verison) throws NiFiClientException, IOException;
+
+    ConnectionEntity createConnection(String parentGroupId, ConnectionEntity connectionEntity) throws NiFiClientException, IOException;
+
+    ConnectionEntity updateConnection(ConnectionEntity connectionEntity) throws NiFiClientException, IOException;
+
+    DropRequestEntity emptyQueue(String connectionId) throws NiFiClientException, IOException;
+
+    DropRequestEntity getDropRequest(String connectionId, String dropRequestId) throws NiFiClientException, IOException;
+
+    DropRequestEntity deleteDropRequest(String connectionId, String dropRequestId) throws NiFiClientException, IOException;
+}
diff --git a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/ControllerServicesClient.java b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/ControllerServicesClient.java
index af55421..71e6e71 100644
--- a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/ControllerServicesClient.java
+++ b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/ControllerServicesClient.java
@@ -17,7 +17,9 @@
 package org.apache.nifi.toolkit.cli.impl.client.nifi;
 
 import org.apache.nifi.web.api.entity.ControllerServiceEntity;
+import org.apache.nifi.web.api.entity.ControllerServiceReferencingComponentsEntity;
 import org.apache.nifi.web.api.entity.ControllerServiceRunStatusEntity;
+import org.apache.nifi.web.api.entity.UpdateControllerServiceReferenceRequestEntity;
 
 import java.io.IOException;
 
@@ -30,4 +32,13 @@ public interface ControllerServicesClient {
 
     ControllerServiceEntity activateControllerService(String id, ControllerServiceRunStatusEntity runStatusEntity) throws NiFiClientException, IOException;
 
+    ControllerServiceEntity createControllerService(String parentGroupdId, ControllerServiceEntity controllerServiceEntity) throws NiFiClientException, IOException;
+
+    ControllerServiceEntity updateControllerService(ControllerServiceEntity controllerServiceEntity) throws NiFiClientException, IOException;
+
+    ControllerServiceEntity deleteControllerService(ControllerServiceEntity controllerServiceEntity) throws NiFiClientException, IOException;
+
+    ControllerServiceReferencingComponentsEntity getControllerServiceReferences(String id) throws NiFiClientException, IOException;
+
+    ControllerServiceReferencingComponentsEntity updateControllerServiceReferences(UpdateControllerServiceReferenceRequestEntity referencesEntity) throws NiFiClientException, IOException;
 }
diff --git a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/ControllerServicesClient.java b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/CountersClient.java
similarity index 63%
copy from nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/ControllerServicesClient.java
copy to nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/CountersClient.java
index af55421..d886cbb 100644
--- a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/ControllerServicesClient.java
+++ b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/CountersClient.java
@@ -16,18 +16,11 @@
  */
 package org.apache.nifi.toolkit.cli.impl.client.nifi;
 
-import org.apache.nifi.web.api.entity.ControllerServiceEntity;
-import org.apache.nifi.web.api.entity.ControllerServiceRunStatusEntity;
+import org.apache.nifi.web.api.entity.CountersEntity;
 
 import java.io.IOException;
 
-/**
- * Client for interacting with NiFi's Controller Services Resource.
- */
-public interface ControllerServicesClient {
-
-    ControllerServiceEntity getControllerService(String id) throws NiFiClientException, IOException;
-
-    ControllerServiceEntity activateControllerService(String id, ControllerServiceRunStatusEntity runStatusEntity) throws NiFiClientException, IOException;
+public interface CountersClient {
 
+    CountersEntity getCounters() throws NiFiClientException, IOException;
 }
diff --git a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/FlowClient.java b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/FlowClient.java
index 6cd60d5..8980d6e 100644
--- a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/FlowClient.java
+++ b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/FlowClient.java
@@ -18,6 +18,7 @@ package org.apache.nifi.toolkit.cli.impl.client.nifi;
 
 import org.apache.nifi.web.api.entity.ActivateControllerServicesEntity;
 import org.apache.nifi.web.api.entity.ClusteSummaryEntity;
+import org.apache.nifi.web.api.entity.ConnectionStatusEntity;
 import org.apache.nifi.web.api.entity.ControllerServicesEntity;
 import org.apache.nifi.web.api.entity.CurrentUserEntity;
 import org.apache.nifi.web.api.entity.ProcessGroupFlowEntity;
@@ -127,4 +128,11 @@ public interface FlowClient {
      */
     TemplatesEntity getTemplates() throws NiFiClientException, IOException;
 
+    /**
+     * Retrives the status for the connection with the given ID
+     * @param connectionId the id of the connection
+     * @param nodewise whether or not node-wise information should be returned
+     * @return the status for the connection
+     */
+    ConnectionStatusEntity getConnectionStatus(String connectionId, boolean nodewise) throws NiFiClientException, IOException;
 }
diff --git a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/ControllerServicesClient.java b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/InputPortClient.java
similarity index 58%
copy from nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/ControllerServicesClient.java
copy to nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/InputPortClient.java
index af55421..ffbfca3 100644
--- a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/ControllerServicesClient.java
+++ b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/InputPortClient.java
@@ -16,18 +16,20 @@
  */
 package org.apache.nifi.toolkit.cli.impl.client.nifi;
 
-import org.apache.nifi.web.api.entity.ControllerServiceEntity;
-import org.apache.nifi.web.api.entity.ControllerServiceRunStatusEntity;
+import org.apache.nifi.web.api.entity.PortEntity;
 
 import java.io.IOException;
 
-/**
- * Client for interacting with NiFi's Controller Services Resource.
- */
-public interface ControllerServicesClient {
+public interface InputPortClient {
+    PortEntity createInputPort(String parentGroupId, PortEntity entity) throws NiFiClientException, IOException;
+
+    PortEntity getInputPort(String id) throws NiFiClientException, IOException;
+
+    PortEntity updateInputPort(PortEntity entity) throws NiFiClientException, IOException;
 
-    ControllerServiceEntity getControllerService(String id) throws NiFiClientException, IOException;
+    PortEntity deleteInputPort(PortEntity entity) throws NiFiClientException, IOException;
 
-    ControllerServiceEntity activateControllerService(String id, ControllerServiceRunStatusEntity runStatusEntity) throws NiFiClientException, IOException;
+    PortEntity startInputPort(PortEntity entity) throws NiFiClientException, IOException;
 
+    PortEntity stopInputPort(PortEntity entity) throws NiFiClientException, IOException;
 }
diff --git a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/NiFiClient.java b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/NiFiClient.java
index dfe16f7..3c66fe3 100644
--- a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/NiFiClient.java
+++ b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/NiFiClient.java
@@ -90,6 +90,13 @@ public interface NiFiClient extends Closeable {
     ProcessGroupClient getProcessGroupClientForToken(String token);
 
     // ----- VersionsClient -----
+    ProcessorClient getProcessorClient();
+
+    ProcessorClient getProcessorClientForProxiedEntities(String... proxiedEntity);
+
+    ProcessorClient getProcessorClientForToken(String token);
+
+    // ----- VersionsClient -----
 
     VersionsClient getVersionsClient();
 
@@ -137,6 +144,47 @@ public interface NiFiClient extends Closeable {
 
     ParamContextClient getParamContextClientForToken(String token);
 
+    // ----- ParamContextClient -----
+
+    CountersClient getCountersClient();
+
+    CountersClient getCountersClientForProxiedEntities(String ... proxiedEntity);
+
+    CountersClient getCountersClientForToken(String token);
+
+    // ----- ConnectionClient -----
+
+    ConnectionClient getConnectionClient();
+
+    ConnectionClient getConnectionClientForProxiedEntities(String... proxiedEntity);
+
+    ConnectionClient getConnectionClientForToken(String token);
+
+    // ----- RemoteProcessGroupClient -----
+
+    RemoteProcessGroupClient getRemoteProcessGroupClient();
+
+    RemoteProcessGroupClient getRemoteProcessGroupClientForProxiedEntities(String... proxiedEntity);
+
+    RemoteProcessGroupClient getRemoteProcessGroupClientForToken(String token);
+
+    // ----- InputPortClient -----
+
+    InputPortClient getInputPortClient();
+
+    InputPortClient getInputPortClientForProxiedEntities(String... proxiedEntity);
+
+    InputPortClient getInputPortClientForToken(String token);
+
+    // ----- OutputPortClient -----
+
+    OutputPortClient getOutputPortClient();
+
+    OutputPortClient getOutputPortClientForProxiedEntities(String... proxiedEntity);
+
+    OutputPortClient getOutputPortClientForToken(String token);
+
+
     /**
      * The builder interface that implementations should provide for obtaining the client.
      */
diff --git a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/ControllerServicesClient.java b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/OutputPortClient.java
similarity index 59%
copy from nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/ControllerServicesClient.java
copy to nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/OutputPortClient.java
index af55421..b635404 100644
--- a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/ControllerServicesClient.java
+++ b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/OutputPortClient.java
@@ -16,18 +16,21 @@
  */
 package org.apache.nifi.toolkit.cli.impl.client.nifi;
 
-import org.apache.nifi.web.api.entity.ControllerServiceEntity;
-import org.apache.nifi.web.api.entity.ControllerServiceRunStatusEntity;
+import org.apache.nifi.web.api.entity.PortEntity;
 
 import java.io.IOException;
 
-/**
- * Client for interacting with NiFi's Controller Services Resource.
- */
-public interface ControllerServicesClient {
+public interface OutputPortClient {
+
+    PortEntity createOutputPort(String parentGroupId, PortEntity entity) throws NiFiClientException, IOException;
+
+    PortEntity getOutputPort(String id) throws NiFiClientException, IOException;
+
+    PortEntity updateOutputPort(PortEntity entity) throws NiFiClientException, IOException;
 
-    ControllerServiceEntity getControllerService(String id) throws NiFiClientException, IOException;
+    PortEntity deleteOutputPort(PortEntity entity) throws NiFiClientException, IOException;
 
-    ControllerServiceEntity activateControllerService(String id, ControllerServiceRunStatusEntity runStatusEntity) throws NiFiClientException, IOException;
+    PortEntity startInpuOutputPort(PortEntity entity) throws NiFiClientException, IOException;
 
+    PortEntity stopOutputPort(PortEntity entity) throws NiFiClientException, IOException;
 }
diff --git a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/ProcessGroupClient.java b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/ProcessGroupClient.java
index 47aea07..5843d20 100644
--- a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/ProcessGroupClient.java
+++ b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/ProcessGroupClient.java
@@ -53,5 +53,4 @@ public interface ProcessGroupClient {
             throws NiFiClientException, IOException;
 
     TemplateEntity uploadTemplate(String processGroupId, TemplateDTO templateDTO) throws NiFiClientException, IOException;
-
 }
diff --git a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/ProcessorClient.java b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/ProcessorClient.java
new file mode 100644
index 0000000..bcce57c
--- /dev/null
+++ b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/ProcessorClient.java
@@ -0,0 +1,45 @@
+/*
+ * 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.
+ */
+package org.apache.nifi.toolkit.cli.impl.client.nifi;
+
+import org.apache.nifi.web.api.entity.ProcessorEntity;
+
+import java.io.IOException;
+
+public interface ProcessorClient {
+    ProcessorEntity createProcessor(String parentGroupdId, ProcessorEntity processorEntity) throws NiFiClientException, IOException;
+
+    ProcessorEntity getProcessor(String processorId) throws NiFiClientException, IOException;
+
+    ProcessorEntity updateProcessor(ProcessorEntity entity) throws NiFiClientException, IOException;
+
+    ProcessorEntity startProcessor(String processorId, String clientId, long version) throws NiFiClientException, IOException;
+
+    ProcessorEntity startProcessor(ProcessorEntity processorEntity) throws NiFiClientException, IOException;
+
+    ProcessorEntity stopProcessor(String processorId, String clientId, long version) throws NiFiClientException, IOException;
+
+    ProcessorEntity stopProcessor(ProcessorEntity processorEntity) throws NiFiClientException, IOException;
+
+    ProcessorEntity disableProcessor(String processorId, String clientId, long version) throws NiFiClientException, IOException;
+
+    ProcessorEntity disableProcessor(ProcessorEntity processorEntity) throws NiFiClientException, IOException;
+
+    ProcessorEntity deleteProcessor(String processorId, String clientId, long version) throws NiFiClientException, IOException;
+
+    ProcessorEntity deleteProcessor(ProcessorEntity processorEntity) throws NiFiClientException, IOException;
+}
diff --git a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/ControllerServicesClient.java b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/RemoteProcessGroupClient.java
similarity index 50%
copy from nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/ControllerServicesClient.java
copy to nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/RemoteProcessGroupClient.java
index af55421..3c0b0ba 100644
--- a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/ControllerServicesClient.java
+++ b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/RemoteProcessGroupClient.java
@@ -16,18 +16,20 @@
  */
 package org.apache.nifi.toolkit.cli.impl.client.nifi;
 
-import org.apache.nifi.web.api.entity.ControllerServiceEntity;
-import org.apache.nifi.web.api.entity.ControllerServiceRunStatusEntity;
+import org.apache.nifi.web.api.entity.RemoteProcessGroupEntity;
 
 import java.io.IOException;
 
-/**
- * Client for interacting with NiFi's Controller Services Resource.
- */
-public interface ControllerServicesClient {
+public interface RemoteProcessGroupClient {
+    RemoteProcessGroupEntity createRemoteProcessGroup(String parentGroupId, RemoteProcessGroupEntity entity) throws NiFiClientException, IOException;
+
+    RemoteProcessGroupEntity getRemoteProcessGroup(String id) throws NiFiClientException, IOException;
+
+    RemoteProcessGroupEntity updateRemoteProcessGroup(RemoteProcessGroupEntity entity) throws NiFiClientException, IOException;
 
-    ControllerServiceEntity getControllerService(String id) throws NiFiClientException, IOException;
+    RemoteProcessGroupEntity deleteRemoteProcessGroup(RemoteProcessGroupEntity entity) throws NiFiClientException, IOException;
 
-    ControllerServiceEntity activateControllerService(String id, ControllerServiceRunStatusEntity runStatusEntity) throws NiFiClientException, IOException;
+    RemoteProcessGroupEntity startTransmitting(RemoteProcessGroupEntity entity) throws NiFiClientException, IOException;
 
+    RemoteProcessGroupEntity stopTransmitting(RemoteProcessGroupEntity entity) throws NiFiClientException, IOException;
 }
diff --git a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/impl/CRUDJerseyClient.java b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/impl/CRUDJerseyClient.java
new file mode 100644
index 0000000..b333427
--- /dev/null
+++ b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/impl/CRUDJerseyClient.java
@@ -0,0 +1,119 @@
+/*
+ * 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.
+ */
+package org.apache.nifi.toolkit.cli.impl.client.nifi.impl;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientException;
+import org.apache.nifi.web.api.dto.RevisionDTO;
+import org.apache.nifi.web.api.entity.ComponentEntity;
+
+import javax.ws.rs.client.Entity;
+import javax.ws.rs.client.WebTarget;
+import javax.ws.rs.core.MediaType;
+import java.io.IOException;
+import java.util.Map;
+
+public class CRUDJerseyClient<T extends ComponentEntity> extends AbstractJerseyClient {
+    private final WebTarget creationTarget;
+    private final WebTarget accessTarget;
+    private final Class<T> entityType;
+    private final String componentType;
+
+
+    public CRUDJerseyClient(final WebTarget creationTarget, final WebTarget accessTarget, final Map<String, String> headers, final Class<T> entityType, final String componentType) {
+        super(headers);
+        this.entityType = entityType;
+        this.componentType = componentType;
+
+        this.creationTarget = creationTarget;
+        this.accessTarget = accessTarget;
+    }
+
+
+    protected T createComponent(final String parentGroupdId, final T entity) throws NiFiClientException, IOException {
+        if (StringUtils.isBlank(parentGroupdId)) {
+            throw new IllegalArgumentException("Parent process group id cannot be null or blank");
+        }
+
+        if (entity == null) {
+            throw new IllegalArgumentException("Entity cannot be null");
+        }
+
+        return executeAction("Error creating " + componentType, () -> {
+            final WebTarget target = creationTarget
+                .resolveTemplate("pgId", parentGroupdId);
+
+            return getRequestBuilder(target).post(
+                Entity.entity(entity, MediaType.APPLICATION_JSON_TYPE),
+                entityType
+            );
+        });
+    }
+
+
+    protected T getComponent(final String id) throws NiFiClientException, IOException {
+        if (StringUtils.isBlank(id)) {
+            throw new IllegalArgumentException(componentType + " ID cannot be null");
+        }
+
+        return executeAction("Error retrieving status of " + componentType, () -> {
+            final WebTarget target = accessTarget.resolveTemplate("id", id);
+            return getRequestBuilder(target).get(entityType);
+        });
+    }
+
+
+    protected T updateComponent(final T entity) throws NiFiClientException, IOException {
+        if (entity == null) {
+            throw new IllegalArgumentException(componentType + " entity cannot be null");
+        }
+
+        return executeAction("Error updating " + componentType, () -> {
+            final WebTarget target = accessTarget.resolveTemplate("id", entity.getId());
+
+            return getRequestBuilder(target).put(
+                Entity.entity(entity, MediaType.APPLICATION_JSON_TYPE),
+                entityType
+            );
+        });
+    }
+
+    public T deleteComponent(final T entity) throws NiFiClientException, IOException {
+        if (entity == null) {
+            throw new IllegalArgumentException(componentType + " entity cannot be null");
+        }
+
+        if (entity.getId() == null) {
+            throw new IllegalArgumentException(componentType + " ID cannot be null");
+        }
+
+        final RevisionDTO revision = entity.getRevision();
+        if (revision == null) {
+            throw new IllegalArgumentException("Revision cannot be null");
+        }
+
+        return executeAction("Error deleting " + componentType, () -> {
+            final WebTarget target = accessTarget
+                .queryParam("version", revision.getVersion())
+                .queryParam("clientId", revision.getClientId())
+                .resolveTemplate("id", entity.getId());
+
+            return getRequestBuilder(target).delete(entityType);
+        });
+    }
+
+}
diff --git a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/impl/JerseyConnectionClient.java b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/impl/JerseyConnectionClient.java
new file mode 100644
index 0000000..51ded4e
--- /dev/null
+++ b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/impl/JerseyConnectionClient.java
@@ -0,0 +1,187 @@
+/*
+ * 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.
+ */
+package org.apache.nifi.toolkit.cli.impl.client.nifi.impl;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.toolkit.cli.impl.client.nifi.ConnectionClient;
+import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientException;
+import org.apache.nifi.web.api.entity.ConnectionEntity;
+import org.apache.nifi.web.api.entity.DropRequestEntity;
+
+import javax.ws.rs.client.Entity;
+import javax.ws.rs.client.WebTarget;
+import javax.ws.rs.core.MediaType;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Map;
+
+public class JerseyConnectionClient extends AbstractJerseyClient implements ConnectionClient {
+    private final WebTarget connectionTarget;
+    private final WebTarget processGroupTarget;
+    private final WebTarget flowFileQueueTarget;
+
+    public JerseyConnectionClient(final WebTarget baseTarget) {
+        this(baseTarget, Collections.emptyMap());
+    }
+
+    public JerseyConnectionClient(final WebTarget baseTarget, final Map<String,String> headers) {
+        super(headers);
+
+        this.connectionTarget = baseTarget.path("/connections/{id}");
+        this.processGroupTarget = baseTarget.path("/process-groups/{pgId}");
+        this.flowFileQueueTarget = baseTarget.path("/flowfile-queues/{id}");
+    }
+
+
+    @Override
+    public ConnectionEntity getConnection(final String id) throws NiFiClientException, IOException {
+        if (StringUtils.isBlank(id)) {
+            throw new IllegalArgumentException("Connection id cannot be null or blank");
+        }
+
+        return executeAction("Error getting connection", () -> {
+            final WebTarget target = connectionTarget
+                .resolveTemplate("id", id);
+
+            return getRequestBuilder(target).get(ConnectionEntity.class);
+        });
+    }
+
+    @Override
+    public ConnectionEntity deleteConnection(final String id, final String clientId, final long version) throws NiFiClientException, IOException {
+        if (id == null) {
+            throw new IllegalArgumentException("Connection id cannot be null");
+        }
+
+        return executeAction("Error deleting Connection", () -> {
+            final WebTarget target = connectionTarget
+                .queryParam("version", version)
+                .queryParam("clientId", clientId)
+                .resolveTemplate("id", id);
+
+            return getRequestBuilder(target).delete(ConnectionEntity.class);
+        });
+    }
+
+    @Override
+    public ConnectionEntity deleteConnection(final ConnectionEntity connectionEntity) throws NiFiClientException, IOException {
+        if (connectionEntity == null) {
+            throw new IllegalArgumentException("Connection Entity cannot be null");
+        }
+        if (connectionEntity.getRevision() == null) {
+            throw new IllegalArgumentException("Revision cannot be null");
+        }
+
+        return deleteConnection(connectionEntity.getId(), connectionEntity.getRevision().getClientId(), connectionEntity.getRevision().getVersion());
+    }
+
+    @Override
+    public ConnectionEntity createConnection(final String parentGroupdId, final ConnectionEntity connectionEntity) throws NiFiClientException, IOException {
+        if (StringUtils.isBlank(parentGroupdId)) {
+            throw new IllegalArgumentException("Parent process group id cannot be null or blank");
+        }
+
+        if (connectionEntity == null) {
+            throw new IllegalArgumentException("Connection entity cannot be null");
+        }
+
+        return executeAction("Error creating Connection", () -> {
+            final WebTarget target = processGroupTarget
+                .path("/connections")
+                .resolveTemplate("pgId", parentGroupdId);
+
+            return getRequestBuilder(target).post(
+                Entity.entity(connectionEntity, MediaType.APPLICATION_JSON_TYPE),
+                ConnectionEntity.class
+            );
+        });
+    }
+
+    @Override
+    public ConnectionEntity updateConnection(final ConnectionEntity connectionEntity) throws NiFiClientException, IOException {
+        if (connectionEntity == null) {
+            throw new IllegalArgumentException("Connection entity cannot be null");
+        }
+
+        return executeAction("Error updating Connection", () -> {
+            final WebTarget target = connectionTarget
+                .resolveTemplate("id", connectionEntity.getId());
+
+            return getRequestBuilder(target).put(
+                Entity.entity(connectionEntity, MediaType.APPLICATION_JSON_TYPE),
+                ConnectionEntity.class
+            );
+        });
+    }
+
+    @Override
+    public DropRequestEntity emptyQueue(final String connectionId) throws NiFiClientException, IOException {
+        if (connectionId == null) {
+            throw new IllegalArgumentException("Connection ID cannot be null");
+        }
+
+        return executeAction("Error empty queue for Connection", () -> {
+            final WebTarget target = flowFileQueueTarget
+                .path("drop-requests")
+                .resolveTemplate("id", connectionId);
+
+            return getRequestBuilder(target).post(
+                Entity.entity(connectionId, MediaType.TEXT_PLAIN),
+                DropRequestEntity.class
+            );
+        });
+    }
+
+    @Override
+    public DropRequestEntity getDropRequest(final String connectionId, final String dropRequestId) throws NiFiClientException, IOException {
+        if (connectionId == null) {
+            throw new IllegalArgumentException("Connection ID cannot be null");
+        }
+        if (dropRequestId == null) {
+            throw new IllegalArgumentException("Drop Request ID cannot be null");
+        }
+
+        return executeAction("Error retrieving Drop Request", () -> {
+            final WebTarget target = flowFileQueueTarget
+                .path("drop-requests/{requestId}")
+                .resolveTemplate("id", connectionId)
+                .resolveTemplate("requestId", dropRequestId);
+
+            return getRequestBuilder(target).get(DropRequestEntity.class);
+        });
+    }
+
+    @Override
+    public DropRequestEntity deleteDropRequest(final String connectionId, final String dropRequestId) throws NiFiClientException, IOException {
+        if (connectionId == null) {
+            throw new IllegalArgumentException("Connection ID cannot be null");
+        }
+        if (dropRequestId == null) {
+            throw new IllegalArgumentException("Drop Request ID cannot be null");
+        }
+
+        return executeAction("Error retrieving Drop Request", () -> {
+            final WebTarget target = flowFileQueueTarget
+                .path("drop-requests/{requestId}")
+                .resolveTemplate("id", connectionId)
+                .resolveTemplate("requestId", dropRequestId);
+
+            return getRequestBuilder(target).delete(DropRequestEntity.class);
+        });
+    }
+
+}
diff --git a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/impl/JerseyControllerServicesClient.java b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/impl/JerseyControllerServicesClient.java
index bf72b75..7906f85 100644
--- a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/impl/JerseyControllerServicesClient.java
+++ b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/impl/JerseyControllerServicesClient.java
@@ -19,15 +19,18 @@ package org.apache.nifi.toolkit.cli.impl.client.nifi.impl;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.toolkit.cli.impl.client.nifi.ControllerServicesClient;
 import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientException;
+import org.apache.nifi.web.api.dto.RevisionDTO;
 import org.apache.nifi.web.api.entity.ControllerServiceEntity;
+import org.apache.nifi.web.api.entity.ControllerServiceReferencingComponentsEntity;
 import org.apache.nifi.web.api.entity.ControllerServiceRunStatusEntity;
+import org.apache.nifi.web.api.entity.UpdateControllerServiceReferenceRequestEntity;
 
-import java.io.IOException;
-import java.util.Collections;
-import java.util.Map;
 import javax.ws.rs.client.Entity;
 import javax.ws.rs.client.WebTarget;
 import javax.ws.rs.core.MediaType;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Map;
 
 /**
  * Jersey implementation of ControllerServicersClient.
@@ -35,6 +38,7 @@ import javax.ws.rs.core.MediaType;
 public class JerseyControllerServicesClient extends AbstractJerseyClient implements ControllerServicesClient {
 
     private final WebTarget controllerServicesTarget;
+    private final WebTarget processGroupTarget;
 
     public JerseyControllerServicesClient(final WebTarget baseTarget) {
         this(baseTarget, Collections.emptyMap());
@@ -43,6 +47,7 @@ public class JerseyControllerServicesClient extends AbstractJerseyClient impleme
     public JerseyControllerServicesClient(final WebTarget baseTarget, final Map<String, String> headers) {
         super(headers);
         this.controllerServicesTarget = baseTarget.path("/controller-services");
+        this.processGroupTarget = baseTarget.path("/process-groups/{pgId}");
     }
 
     @Override
@@ -76,4 +81,99 @@ public class JerseyControllerServicesClient extends AbstractJerseyClient impleme
                 ControllerServiceEntity.class);
         });
     }
+
+    @Override
+    public ControllerServiceEntity createControllerService(final String parentGroupdId, final ControllerServiceEntity controllerServiceEntity) throws NiFiClientException, IOException {
+        if (StringUtils.isBlank(parentGroupdId)) {
+            throw new IllegalArgumentException("Parent process group id cannot be null or blank");
+        }
+
+        if (controllerServiceEntity == null) {
+            throw new IllegalArgumentException("Controller Service entity cannot be null");
+        }
+
+        return executeAction("Error creating Controller Service", () -> {
+            final WebTarget target = processGroupTarget
+                .path("/controller-services")
+                .resolveTemplate("pgId", parentGroupdId);
+
+            return getRequestBuilder(target).post(
+                Entity.entity(controllerServiceEntity, MediaType.APPLICATION_JSON_TYPE),
+                ControllerServiceEntity.class
+            );
+        });
+    }
+
+    @Override
+    public ControllerServiceEntity updateControllerService(final ControllerServiceEntity controllerServiceEntity) throws NiFiClientException, IOException {
+        if (controllerServiceEntity == null) {
+            throw new IllegalArgumentException("Controller Service entity cannot be null");
+        }
+
+        return executeAction("Error updating Controller Service", () -> {
+            final WebTarget target = controllerServicesTarget
+                .path("/{id}")
+                .resolveTemplate("id", controllerServiceEntity.getId());
+
+            return getRequestBuilder(target).put(
+                Entity.entity(controllerServiceEntity, MediaType.APPLICATION_JSON_TYPE),
+                ControllerServiceEntity.class
+            );
+        });
+    }
+
+    @Override
+    public ControllerServiceEntity deleteControllerService(final ControllerServiceEntity controllerServiceEntity) throws NiFiClientException, IOException {
+        if (controllerServiceEntity == null) {
+            throw new IllegalArgumentException("Controller Service entity cannot be null");
+        }
+
+        final RevisionDTO revision = controllerServiceEntity.getRevision();
+        if (revision == null) {
+            throw new IllegalArgumentException("Controller Service Revision cannot be null");
+        }
+
+        return executeAction("Error deleting Controller Service", () -> {
+            final WebTarget target = controllerServicesTarget
+                .path("/{id}")
+                .queryParam("version", revision.getVersion())
+                .queryParam("clientId", revision.getClientId())
+                .resolveTemplate("id", controllerServiceEntity.getId());
+
+            return getRequestBuilder(target).delete(ControllerServiceEntity.class);
+        });
+    }
+
+    @Override
+    public ControllerServiceReferencingComponentsEntity getControllerServiceReferences(final String id) throws NiFiClientException, IOException {
+        if (StringUtils.isBlank(id)) {
+            throw new IllegalArgumentException("Controller service id cannot be null");
+        }
+
+        return executeAction("Error retrieving Controller Service's referencing components", () -> {
+            final WebTarget target = controllerServicesTarget.path("{id}/references").resolveTemplate("id", id);
+            return getRequestBuilder(target).get(ControllerServiceReferencingComponentsEntity.class);
+        });
+    }
+
+    @Override
+    public ControllerServiceReferencingComponentsEntity updateControllerServiceReferences(final UpdateControllerServiceReferenceRequestEntity referencesEntity)
+        throws NiFiClientException, IOException {
+
+        if (referencesEntity == null) {
+            throw new IllegalArgumentException("Controller Service references entity cannot be null");
+        }
+
+        return executeAction("Error updating Controller Service references", () -> {
+            final WebTarget target = controllerServicesTarget
+                .path("/{id}/references")
+                .resolveTemplate("id", referencesEntity.getId());
+
+            return getRequestBuilder(target).put(
+                Entity.entity(referencesEntity, MediaType.APPLICATION_JSON_TYPE),
+                ControllerServiceReferencingComponentsEntity.class
+            );
+        });
+
+    }
 }
diff --git a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/impl/JerseyCountersClient.java b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/impl/JerseyCountersClient.java
new file mode 100644
index 0000000..81b2bd7
--- /dev/null
+++ b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/impl/JerseyCountersClient.java
@@ -0,0 +1,47 @@
+/*
+ * 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.
+ */
+package org.apache.nifi.toolkit.cli.impl.client.nifi.impl;
+
+import org.apache.nifi.toolkit.cli.impl.client.nifi.CountersClient;
+import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientException;
+import org.apache.nifi.web.api.entity.CountersEntity;
+
+import javax.ws.rs.client.WebTarget;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Map;
+
+public class JerseyCountersClient extends AbstractJerseyClient implements CountersClient {
+    private final WebTarget countersTarget;
+
+    public JerseyCountersClient(final WebTarget baseTarget) {
+        this(baseTarget, Collections.emptyMap());
+    }
+
+    public JerseyCountersClient(final WebTarget baseTarget, final Map<String,String> headers) {
+        super(headers);
+        this.countersTarget = baseTarget.path("/counters");
+    }
+
+    @Override
+    public CountersEntity getCounters() throws NiFiClientException, IOException {
+        return executeAction("Error retrieving counters", () -> {
+            return getRequestBuilder(countersTarget).get(CountersEntity.class);
+        });
+
+    }
+}
diff --git a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/impl/JerseyFlowClient.java b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/impl/JerseyFlowClient.java
index 30add12..f18f094 100644
--- a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/impl/JerseyFlowClient.java
+++ b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/impl/JerseyFlowClient.java
@@ -26,6 +26,7 @@ import org.apache.nifi.web.api.dto.flow.ProcessGroupFlowDTO;
 import org.apache.nifi.web.api.entity.ActivateControllerServicesEntity;
 import org.apache.nifi.web.api.entity.ClusteSummaryEntity;
 import org.apache.nifi.web.api.entity.ComponentEntity;
+import org.apache.nifi.web.api.entity.ConnectionStatusEntity;
 import org.apache.nifi.web.api.entity.ControllerServicesEntity;
 import org.apache.nifi.web.api.entity.CurrentUserEntity;
 import org.apache.nifi.web.api.entity.ProcessGroupFlowEntity;
@@ -252,4 +253,15 @@ public class JerseyFlowClient extends AbstractJerseyClient implements FlowClient
             return getRequestBuilder(target).get(TemplatesEntity.class);
         });
     }
+
+    @Override
+    public ConnectionStatusEntity getConnectionStatus(final String connectionId, final boolean nodewise) throws NiFiClientException, IOException {
+        return executeAction("Error retrieving Connection status", () -> {
+            final WebTarget target = flowTarget.path("/connections/{connectionId}/status")
+                .resolveTemplate("connectionId", connectionId)
+                .queryParam("nodewise", nodewise);
+
+            return getRequestBuilder(target).get(ConnectionStatusEntity.class);
+        });
+    }
 }
diff --git a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/impl/JerseyInputPortClient.java b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/impl/JerseyInputPortClient.java
new file mode 100644
index 0000000..6c10a69
--- /dev/null
+++ b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/impl/JerseyInputPortClient.java
@@ -0,0 +1,88 @@
+/*
+ * 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.
+ */
+package org.apache.nifi.toolkit.cli.impl.client.nifi.impl;
+
+import org.apache.nifi.toolkit.cli.impl.client.nifi.InputPortClient;
+import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientException;
+import org.apache.nifi.web.api.dto.PortDTO;
+import org.apache.nifi.web.api.entity.PortEntity;
+
+import javax.ws.rs.client.WebTarget;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Map;
+
+public class JerseyInputPortClient extends CRUDJerseyClient<PortEntity> implements InputPortClient {
+
+    public JerseyInputPortClient(final WebTarget baseTarget) {
+        this(baseTarget, Collections.emptyMap());
+    }
+
+    public JerseyInputPortClient(final WebTarget baseTarget, final Map<String, String> headers) {
+        super(baseTarget.path("/process-groups/{pgId}/input-ports"),
+            baseTarget.path("/input-ports/{id}"),
+            headers,
+            PortEntity.class,
+            "Input Port");
+    }
+
+    @Override
+    public PortEntity createInputPort(final String parentGroupId, final PortEntity entity) throws NiFiClientException, IOException {
+        return createComponent(parentGroupId, entity);
+    }
+
+    @Override
+    public PortEntity getInputPort(final String id) throws NiFiClientException, IOException {
+        return getComponent(id);
+    }
+
+    @Override
+    public PortEntity updateInputPort(final PortEntity entity) throws NiFiClientException, IOException {
+        return updateComponent(entity);
+    }
+
+    @Override
+    public PortEntity deleteInputPort(final PortEntity entity) throws NiFiClientException, IOException {
+        return deleteComponent(entity);
+    }
+
+    @Override
+    public PortEntity startInputPort(final PortEntity entity) throws NiFiClientException, IOException {
+        final PortEntity startEntity = createStateEntity(entity, "RUNNING");
+        return updateInputPort(startEntity);
+    }
+
+    @Override
+    public PortEntity stopInputPort(final PortEntity entity) throws NiFiClientException, IOException {
+        final PortEntity startEntity = createStateEntity(entity, "STOPPED");
+        return updateInputPort(startEntity);
+    }
+
+    private PortEntity createStateEntity(final PortEntity entity, final String state) {
+        final PortDTO component = new PortDTO();
+        component.setId(entity.getComponent().getId());
+        component.setParentGroupId(entity.getComponent().getParentGroupId());
+        component.setState(state);
+
+        final PortEntity stateEntity = new PortEntity();
+        stateEntity.setId(entity.getId());
+        stateEntity.setRevision(entity.getRevision());
+        stateEntity.setComponent(component);
+
+        return stateEntity;
+    }
+}
diff --git a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/impl/JerseyNiFiClient.java b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/impl/JerseyNiFiClient.java
index c91067a..fbc71e6 100644
--- a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/impl/JerseyNiFiClient.java
+++ b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/impl/JerseyNiFiClient.java
@@ -22,14 +22,20 @@ import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.module.jaxb.JaxbAnnotationIntrospector;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.registry.security.util.ProxiedEntitiesUtils;
+import org.apache.nifi.toolkit.cli.impl.client.nifi.ConnectionClient;
 import org.apache.nifi.toolkit.cli.impl.client.nifi.ControllerClient;
 import org.apache.nifi.toolkit.cli.impl.client.nifi.ControllerServicesClient;
+import org.apache.nifi.toolkit.cli.impl.client.nifi.CountersClient;
 import org.apache.nifi.toolkit.cli.impl.client.nifi.FlowClient;
+import org.apache.nifi.toolkit.cli.impl.client.nifi.InputPortClient;
 import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClient;
 import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientConfig;
+import org.apache.nifi.toolkit.cli.impl.client.nifi.OutputPortClient;
 import org.apache.nifi.toolkit.cli.impl.client.nifi.ParamContextClient;
 import org.apache.nifi.toolkit.cli.impl.client.nifi.PoliciesClient;
 import org.apache.nifi.toolkit.cli.impl.client.nifi.ProcessGroupClient;
+import org.apache.nifi.toolkit.cli.impl.client.nifi.ProcessorClient;
+import org.apache.nifi.toolkit.cli.impl.client.nifi.RemoteProcessGroupClient;
 import org.apache.nifi.toolkit.cli.impl.client.nifi.ReportingTasksClient;
 import org.apache.nifi.toolkit.cli.impl.client.nifi.TemplatesClient;
 import org.apache.nifi.toolkit.cli.impl.client.nifi.TenantsClient;
@@ -43,7 +49,6 @@ import javax.net.ssl.SSLContext;
 import javax.ws.rs.client.Client;
 import javax.ws.rs.client.ClientBuilder;
 import javax.ws.rs.client.WebTarget;
-import java.io.IOException;
 import java.net.URI;
 import java.util.Arrays;
 import java.util.HashMap;
@@ -184,6 +189,23 @@ public class JerseyNiFiClient implements NiFiClient {
     }
 
     @Override
+    public ProcessorClient getProcessorClient() {
+        return new JerseyProcessorClient(baseTarget);
+    }
+
+    @Override
+    public ProcessorClient getProcessorClientForProxiedEntities(final String... proxiedEntity) {
+        final Map<String,String> headers = getHeaders(proxiedEntity);
+        return new JerseyProcessorClient(baseTarget, headers);
+    }
+
+    @Override
+    public ProcessorClient getProcessorClientForToken(final String token) {
+        final Map<String,String> headers = getHeadersWithToken(token);
+        return new JerseyProcessorClient(baseTarget, headers);
+    }
+
+    @Override
     public VersionsClient getVersionsClient() {
         return new JerseyVersionsClient(baseTarget);
     }
@@ -286,7 +308,92 @@ public class JerseyNiFiClient implements NiFiClient {
     }
 
     @Override
-    public void close() throws IOException {
+    public CountersClient getCountersClient() {
+        return new JerseyCountersClient(baseTarget);
+    }
+
+    @Override
+    public CountersClient getCountersClientForProxiedEntities(final String... proxiedEntity) {
+        final Map<String, String> headers = getHeaders(proxiedEntity);
+        return new JerseyCountersClient(baseTarget, headers);
+    }
+
+    @Override
+    public CountersClient getCountersClientForToken(final String token) {
+        final Map<String, String> headers = getHeadersWithToken(token);
+        return new JerseyCountersClient(baseTarget, headers);
+    }
+
+    @Override
+    public ConnectionClient getConnectionClient() {
+        return new JerseyConnectionClient(baseTarget);
+    }
+
+    @Override
+    public ConnectionClient getConnectionClientForProxiedEntities(final String... proxiedEntity) {
+        final Map<String, String> headers = getHeaders(proxiedEntity);
+        return new JerseyConnectionClient(baseTarget, headers);
+    }
+
+    @Override
+    public ConnectionClient getConnectionClientForToken(final String token) {
+        final Map<String, String> headers = getHeadersWithToken(token);
+        return new JerseyConnectionClient(baseTarget, headers);
+    }
+
+    @Override
+    public RemoteProcessGroupClient getRemoteProcessGroupClient() {
+        return new JerseyRemoteProcessGroupClient(baseTarget);
+    }
+
+    @Override
+    public RemoteProcessGroupClient getRemoteProcessGroupClientForProxiedEntities(final String... proxiedEntity) {
+        final Map<String, String> headers = getHeaders(proxiedEntity);
+        return new JerseyRemoteProcessGroupClient(baseTarget, headers);
+    }
+
+    @Override
+    public RemoteProcessGroupClient getRemoteProcessGroupClientForToken(final String token) {
+        final Map<String, String> headers = getHeadersWithToken(token);
+        return new JerseyRemoteProcessGroupClient(baseTarget, headers);
+    }
+
+    @Override
+    public InputPortClient getInputPortClient() {
+        return new JerseyInputPortClient(baseTarget);
+    }
+
+    @Override
+    public InputPortClient getInputPortClientForProxiedEntities(final String... proxiedEntity) {
+        final Map<String, String> headers = getHeaders(proxiedEntity);
+        return new JerseyInputPortClient(baseTarget, headers);
+    }
+
+    @Override
+    public InputPortClient getInputPortClientForToken(final String token) {
+        final Map<String, String> headers = getHeadersWithToken(token);
+        return new JerseyInputPortClient(baseTarget, headers);
+    }
+
+    @Override
+    public OutputPortClient getOutputPortClient() {
+        return new JerseyOutputPortClient(baseTarget);
+    }
+
+    @Override
+    public OutputPortClient getOutputPortClientForProxiedEntities(final String... proxiedEntity) {
+        final Map<String, String> headers = getHeaders(proxiedEntity);
+        return new JerseyOutputPortClient(baseTarget, headers);
+    }
+
+    @Override
+    public OutputPortClient getOutputPortClientForToken(final String token) {
+        final Map<String, String> headers = getHeadersWithToken(token);
+        return new JerseyOutputPortClient(baseTarget, headers);
+    }
+
+    @Override
+    public void close() {
         if (this.client != null) {
             try {
                 this.client.close();
diff --git a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/impl/JerseyOutputPortClient.java b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/impl/JerseyOutputPortClient.java
new file mode 100644
index 0000000..46637fc
--- /dev/null
+++ b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/impl/JerseyOutputPortClient.java
@@ -0,0 +1,88 @@
+/*
+ * 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.
+ */
+package org.apache.nifi.toolkit.cli.impl.client.nifi.impl;
+
+import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientException;
+import org.apache.nifi.toolkit.cli.impl.client.nifi.OutputPortClient;
+import org.apache.nifi.web.api.dto.PortDTO;
+import org.apache.nifi.web.api.entity.PortEntity;
+
+import javax.ws.rs.client.WebTarget;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Map;
+
+public class JerseyOutputPortClient extends CRUDJerseyClient<PortEntity> implements OutputPortClient {
+
+    public JerseyOutputPortClient(final WebTarget baseTarget) {
+        this(baseTarget, Collections.emptyMap());
+    }
+
+    public JerseyOutputPortClient(final WebTarget baseTarget, final Map<String, String> headers) {
+        super(baseTarget.path("/process-groups/{pgId}/output-ports"),
+            baseTarget.path("/output-ports/{id}"),
+            headers,
+            PortEntity.class,
+            "Output Port");
+    }
+
+    @Override
+    public PortEntity createOutputPort(final String parentGroupId, final PortEntity entity) throws NiFiClientException, IOException {
+        return createComponent(parentGroupId, entity);
+    }
+
+    @Override
+    public PortEntity getOutputPort(final String id) throws NiFiClientException, IOException {
+        return getComponent(id);
+    }
+
+    @Override
+    public PortEntity updateOutputPort(final PortEntity entity) throws NiFiClientException, IOException {
+        return updateComponent(entity);
+    }
+
+    @Override
+    public PortEntity deleteOutputPort(final PortEntity entity) throws NiFiClientException, IOException {
+        return deleteComponent(entity);
+    }
+
+    @Override
+    public PortEntity startInpuOutputPort(final PortEntity entity) throws NiFiClientException, IOException {
+        final PortEntity startEntity = createStateEntity(entity, "RUNNING");
+        return updateOutputPort(startEntity);
+    }
+
+    @Override
+    public PortEntity stopOutputPort(final PortEntity entity) throws NiFiClientException, IOException {
+        final PortEntity startEntity = createStateEntity(entity, "STOPPED");
+        return updateOutputPort(startEntity);
+    }
+
+    private PortEntity createStateEntity(final PortEntity entity, final String state) {
+        final PortDTO component = new PortDTO();
+        component.setId(entity.getComponent().getId());
+        component.setParentGroupId(entity.getComponent().getParentGroupId());
+        component.setState(state);
+
+        final PortEntity stateEntity = new PortEntity();
+        stateEntity.setId(entity.getId());
+        stateEntity.setRevision(entity.getRevision());
+        stateEntity.setComponent(component);
+
+        return stateEntity;
+    }
+}
diff --git a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/impl/JerseyProcessorClient.java b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/impl/JerseyProcessorClient.java
new file mode 100644
index 0000000..d304c9a
--- /dev/null
+++ b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/impl/JerseyProcessorClient.java
@@ -0,0 +1,172 @@
+/*
+ * 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.
+ */
+package org.apache.nifi.toolkit.cli.impl.client.nifi.impl;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientException;
+import org.apache.nifi.toolkit.cli.impl.client.nifi.ProcessorClient;
+import org.apache.nifi.web.api.dto.RevisionDTO;
+import org.apache.nifi.web.api.entity.ProcessorEntity;
+import org.apache.nifi.web.api.entity.ProcessorRunStatusEntity;
+
+import javax.ws.rs.client.Entity;
+import javax.ws.rs.client.WebTarget;
+import javax.ws.rs.core.MediaType;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Map;
+
+public class JerseyProcessorClient extends AbstractJerseyClient implements ProcessorClient {
+    private final WebTarget processGroupTarget;
+    private final WebTarget processorTarget;
+
+    public JerseyProcessorClient(final WebTarget baseTarget) {
+        this(baseTarget, Collections.emptyMap());
+    }
+
+    public JerseyProcessorClient(final WebTarget baseTarget, final Map<String, String> headers) {
+        super(headers);
+        this.processGroupTarget = baseTarget.path("/process-groups/{pgId}");
+        this.processorTarget = baseTarget.path("/processors/{id}");
+    }
+
+    @Override
+    public ProcessorEntity createProcessor(final String parentGroupdId, final ProcessorEntity processorEntity) throws NiFiClientException, IOException {
+        if (StringUtils.isBlank(parentGroupdId)) {
+            throw new IllegalArgumentException("Parent process group id cannot be null or blank");
+        }
+
+        if (processorEntity == null) {
+            throw new IllegalArgumentException("Processor entity cannot be null");
+        }
+
+        return executeAction("Error creating Processor", () -> {
+            final WebTarget target = processGroupTarget
+                .path("/processors")
+                .resolveTemplate("pgId", parentGroupdId);
+
+            return getRequestBuilder(target).post(
+                Entity.entity(processorEntity, MediaType.APPLICATION_JSON_TYPE),
+                ProcessorEntity.class
+            );
+        });
+    }
+
+    @Override
+    public ProcessorEntity getProcessor(final String processorId) throws NiFiClientException, IOException {
+        if (StringUtils.isBlank(processorId)) {
+            throw new IllegalArgumentException("Processor ID cannot be null");
+        }
+
+        return executeAction("Error retrieving status of Processor", () -> {
+            final WebTarget target = processorTarget.resolveTemplate("id", processorId);
+            return getRequestBuilder(target).get(ProcessorEntity.class);
+        });
+    }
+
+    @Override
+    public ProcessorEntity updateProcessor(final ProcessorEntity entity) throws NiFiClientException, IOException {
+        if (entity == null) {
+            throw new IllegalArgumentException("Processor entity cannot be null");
+        }
+
+        return executeAction("Error updating Processor", () -> {
+            final WebTarget target = processorTarget.resolveTemplate("id", entity.getId());
+
+            return getRequestBuilder(target).put(
+                Entity.entity(entity, MediaType.APPLICATION_JSON_TYPE),
+                ProcessorEntity.class
+            );
+        });
+    }
+
+    @Override
+    public ProcessorEntity startProcessor(final String processorId, final String clientId, final long version) throws NiFiClientException, IOException {
+        return updateProcessorState(processorId, "RUNNING", clientId, version);
+    }
+
+    @Override
+    public ProcessorEntity startProcessor(final ProcessorEntity processorEntity) throws NiFiClientException, IOException {
+        return startProcessor(processorEntity.getId(), processorEntity.getRevision().getClientId(), processorEntity.getRevision().getVersion());
+    }
+
+    @Override
+    public ProcessorEntity stopProcessor(final String processorId, final String clientId, final long version) throws NiFiClientException, IOException {
+        return updateProcessorState(processorId, "STOPPED", clientId, version);
+    }
+
+    @Override
+    public ProcessorEntity stopProcessor(final ProcessorEntity processorEntity) throws NiFiClientException, IOException {
+        return stopProcessor(processorEntity.getId(), processorEntity.getRevision().getClientId(), processorEntity.getRevision().getVersion());
+    }
+
+    @Override
+    public ProcessorEntity disableProcessor(final String processorId, final String clientId, final long version) throws NiFiClientException, IOException {
+        return updateProcessorState(processorId, "DISABLED", clientId, version);
+    }
+
+    @Override
+    public ProcessorEntity disableProcessor(final ProcessorEntity processorEntity) throws NiFiClientException, IOException {
+        return disableProcessor(processorEntity.getId(), processorEntity.getRevision().getClientId(), processorEntity.getRevision().getVersion());
+    }
+
+    @Override
+    public ProcessorEntity deleteProcessor(final String processorId, final String clientId, final long version) throws NiFiClientException, IOException {
+        if (processorId == null) {
+            throw new IllegalArgumentException("Processor ID cannot be null");
+        }
+
+        return executeAction("Error deleting Processor", () -> {
+            final WebTarget target = processorTarget
+                .queryParam("version", version)
+                .queryParam("clientId", clientId)
+                .resolveTemplate("id", processorId);
+
+            return getRequestBuilder(target).delete(ProcessorEntity.class);
+        });
+    }
+
+    @Override
+    public ProcessorEntity deleteProcessor(final ProcessorEntity processorEntity) throws NiFiClientException, IOException {
+        return deleteProcessor(processorEntity.getId(), processorEntity.getRevision().getClientId(), processorEntity.getRevision().getVersion());
+    }
+
+    private ProcessorEntity updateProcessorState(final String processorId, final String desiredState, final String clientId, final long version) throws NiFiClientException, IOException {
+        if (processorId == null) {
+            throw new IllegalArgumentException("Processor ID cannot be null");
+        }
+
+        return executeAction("Error starting Processor", () -> {
+            final WebTarget target = processorTarget
+                .path("/run-status")
+                .resolveTemplate("id", processorId);
+
+            final ProcessorRunStatusEntity runStatusEntity = new ProcessorRunStatusEntity();
+            runStatusEntity.setState(desiredState);
+
+            final RevisionDTO revisionDto = new RevisionDTO();
+            revisionDto.setClientId(clientId);
+            revisionDto.setVersion(version);
+            runStatusEntity.setRevision(revisionDto);
+
+            return getRequestBuilder(target).put(
+                Entity.entity(runStatusEntity, MediaType.APPLICATION_JSON_TYPE),
+                ProcessorEntity.class
+            );
+        });
+    }
+}
diff --git a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/impl/JerseyRemoteProcessGroupClient.java b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/impl/JerseyRemoteProcessGroupClient.java
new file mode 100644
index 0000000..0a8482e
--- /dev/null
+++ b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/nifi/impl/JerseyRemoteProcessGroupClient.java
@@ -0,0 +1,145 @@
+/*
+ * 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.
+ */
+package org.apache.nifi.toolkit.cli.impl.client.nifi.impl;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientException;
+import org.apache.nifi.toolkit.cli.impl.client.nifi.RemoteProcessGroupClient;
+import org.apache.nifi.web.api.dto.RemoteProcessGroupDTO;
+import org.apache.nifi.web.api.dto.RevisionDTO;
+import org.apache.nifi.web.api.entity.RemoteProcessGroupEntity;
+
+import javax.ws.rs.client.Entity;
+import javax.ws.rs.client.WebTarget;
+import javax.ws.rs.core.MediaType;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Map;
+
+public class JerseyRemoteProcessGroupClient extends AbstractJerseyClient implements RemoteProcessGroupClient {
+    private final WebTarget processGroupTarget;
+    private final WebTarget rpgTarget;
+
+    public JerseyRemoteProcessGroupClient(final WebTarget baseTarget) {
+        this(baseTarget, Collections.emptyMap());
+    }
+
+    public JerseyRemoteProcessGroupClient(final WebTarget baseTarget, final Map<String, String> headers) {
+        super(headers);
+        this.processGroupTarget = baseTarget.path("/process-groups/{pgId}");
+        this.rpgTarget = baseTarget.path("/remote-process-groups/{id}");
+    }
+
+    @Override
+    public RemoteProcessGroupEntity createRemoteProcessGroup(final String parentGroupId, final RemoteProcessGroupEntity entity) throws NiFiClientException, IOException {
+        if (StringUtils.isBlank(parentGroupId)) {
+            throw new IllegalArgumentException("Parent process group id cannot be null or blank");
+        }
+
+        if (entity == null) {
+            throw new IllegalArgumentException("Remote Process Group entity cannot be null");
+        }
+
+        return executeAction("Error creating Remote Process Group", () -> {
+            final WebTarget target = processGroupTarget
+                .path("/remote-process-groups")
+                .resolveTemplate("pgId", parentGroupId);
+
+            return getRequestBuilder(target).post(
+                Entity.entity(entity, MediaType.APPLICATION_JSON_TYPE),
+                RemoteProcessGroupEntity.class
+            );
+        });
+    }
+
+    @Override
+    public RemoteProcessGroupEntity getRemoteProcessGroup(final String id) throws NiFiClientException, IOException {
+        if (StringUtils.isBlank(id)) {
+            throw new IllegalArgumentException("Remote Process Group ID cannot be null");
+        }
+
+        return executeAction("Error retrieving status of Remote Process Group", () -> {
+            final WebTarget target = rpgTarget.resolveTemplate("id", id);
+            return getRequestBuilder(target).get(RemoteProcessGroupEntity.class);
+        });
+    }
+
+    @Override
+    public RemoteProcessGroupEntity updateRemoteProcessGroup(final RemoteProcessGroupEntity entity) throws NiFiClientException, IOException {
+        if (entity == null) {
+            throw new IllegalArgumentException("Remote Process Group entity cannot be null");
+        }
+
+        return executeAction("Error updating Remote Process Group", () -> {
+            final WebTarget target = rpgTarget.resolveTemplate("id", entity.getId());
+
+            return getRequestBuilder(target).put(
+                Entity.entity(entity, MediaType.APPLICATION_JSON_TYPE),
+                RemoteProcessGroupEntity.class
+            );
+        });
+    }
+
+    @Override
+    public RemoteProcessGroupEntity deleteRemoteProcessGroup(final RemoteProcessGroupEntity entity) throws NiFiClientException, IOException {
+        if (entity == null) {
+            throw new IllegalArgumentException("Remote Process Group entity cannot be null");
+        }
+
+        if (entity.getId() == null) {
+            throw new IllegalArgumentException("Remote Process Group ID cannot be null");
+        }
+
+        final RevisionDTO revision = entity.getRevision();
+        if (revision == null) {
+            throw new IllegalArgumentException("Revision cannot be null");
+        }
+
+        return executeAction("Error deleting Remote Process Group", () -> {
+            final WebTarget target = rpgTarget
+                .queryParam("version", revision.getVersion())
+                .queryParam("clientId", revision.getClientId())
+                .resolveTemplate("id", entity.getId());
+
+            return getRequestBuilder(target).delete(RemoteProcessGroupEntity.class);
+        });
+    }
+
+    @Override
+    public RemoteProcessGroupEntity startTransmitting(final RemoteProcessGroupEntity entity) throws NiFiClientException, IOException {
+        return updateTransmitting(entity, true);
+    }
+
+    @Override
+    public RemoteProcessGroupEntity stopTransmitting(final RemoteProcessGroupEntity entity) throws NiFiClientException, IOException {
+        return updateTransmitting(entity, false);
+    }
+
+    private RemoteProcessGroupEntity updateTransmitting(final RemoteProcessGroupEntity entity, final boolean transmitting) throws NiFiClientException, IOException {
+        final RemoteProcessGroupDTO component = new RemoteProcessGroupDTO();
+        component.setId(entity.getComponent().getId());
+        component.setParentGroupId(entity.getComponent().getParentGroupId());
+        component.setTransmitting(transmitting);
+
+        final RemoteProcessGroupEntity transmittingEntity = new RemoteProcessGroupEntity();
+        transmittingEntity.setId(entity.getId());
+        transmittingEntity.setRevision(entity.getRevision());
+        transmittingEntity.setComponent(component);
+
+        return updateRemoteProcessGroup(transmittingEntity);
+    }
+}
diff --git a/pom.xml b/pom.xml
index d540027..bc8d9c4 100644
--- a/pom.xml
+++ b/pom.xml
@@ -8,8 +8,7 @@
     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. -->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/xsd/maven-4.0.0.xsd">
+    language governing permissions and limitations under the License. --><project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/xsd/maven-4.0.0.xsd">
     <modelVersion>4.0.0</modelVersion>
     <parent>
         <groupId>org.apache</groupId>
@@ -35,7 +34,8 @@
         <module>nifi-external</module>
         <module>nifi-toolkit</module>
         <module>nifi-docker</module>
-    </modules>
+        <module>nifi-system-tests</module>
+  </modules>
     <url>https://nifi.apache.org</url>
     <organization>
         <name>Apache NiFi Project</name>
@@ -87,7 +87,7 @@
         <maven.compiler.source>1.8</maven.compiler.source>
         <maven.compiler.target>1.8</maven.compiler.target>
         <maven.min-version>3.1.1</maven.min-version>
-        <maven.surefire.arguments />
+        <maven.surefire.arguments/>
         <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
         <project.reporting.outputEncoding>UTF-8</project.reporting.outputEncoding>
         <inceptionYear>2014</inceptionYear>
@@ -581,72 +581,72 @@
                             <!-- Checks for whitespace -->
                             <!-- See http://checkstyle.sf.net/config_whitespace.html -->
                             <module name="FileTabCharacter">
-                                <property name="eachLine" value="true" />
+                                <property name="eachLine" value="true"/>
                             </module>
                             <module name="TreeWalker">
                                 <module name="RegexpSinglelineJava">
-                                    <property name="format" value="\s+$" />
-                                    <property name="message" value="Line has trailing whitespace." />
+                                    <property name="format" value="\s+$"/>
+                                    <property name="message" value="Line has trailing whitespace."/>
                                 </module>
                                 <module name="RegexpSinglelineJava">
-                                    <property name="format" value="[@]see\s+[{][@]link" />
-                                    <property name="message" value="Javadoc @see does not need @link: pick one or the other." />
+                                    <property name="format" value="[@]see\s+[{][@]link"/>
+                                    <property name="message" value="Javadoc @see does not need @link: pick one or the other."/>
                                 </module>
-                                <module name="OuterTypeFilename" />
+                                <module name="OuterTypeFilename"/>
                                 <module name="LineLength">
                                     <!-- needs extra, because Eclipse formatter
                                         ignores the ending left brace -->
-                                    <property name="max" value="200" />
-                                    <property name="ignorePattern" value="^package.*|^import.*|a href|href|http://|https://|ftp://" />
+                                    <property name="max" value="200"/>
+                                    <property name="ignorePattern" value="^package.*|^import.*|a href|href|http://|https://|ftp://"/>
                                 </module>
-                                <module name="AvoidStarImport" />
+                                <module name="AvoidStarImport"/>
                                 <module name="UnusedImports">
-                                    <property name="processJavadoc" value="true" />
+                                    <property name="processJavadoc" value="true"/>
                                 </module>
-                                <module name="NoLineWrap" />
-                                <module name="LeftCurly" />
-                                <module name="RightCurly" />
+                                <module name="NoLineWrap"/>
+                                <module name="LeftCurly"/>
+                                <module name="RightCurly"/>
                                 <module name="RightCurly">
-                                    <property name="option" value="alone_or_singleline" />
-                                    <property name="tokens" value="CLASS_DEF, METHOD_DEF, CTOR_DEF, LITERAL_FOR, LITERAL_WHILE, STATIC_INIT" />
+                                    <property name="option" value="alone_or_singleline"/>
+                                    <property name="tokens" value="CLASS_DEF, METHOD_DEF, CTOR_DEF, LITERAL_FOR, LITERAL_WHILE, STATIC_INIT"/>
                                 </module>
                                 <module name="SeparatorWrap">
-                                    <property name="tokens" value="DOT" />
-                                    <property name="option" value="nl" />
+                                    <property name="tokens" value="DOT"/>
+                                    <property name="option" value="nl"/>
                                 </module>
                                 <module name="SeparatorWrap">
-                                    <property name="tokens" value="COMMA" />
-                                    <property name="option" value="EOL" />
+                                    <property name="tokens" value="COMMA"/>
+                                    <property name="option" value="EOL"/>
                                 </module>
                                 <module name="PackageName">
-                                    <property name="format" value="^[a-z]+(\.[a-z][a-zA-Z0-9]*)*$" />
+                                    <property name="format" value="^[a-z]+(\.[a-z][a-zA-Z0-9]*)*$"/>
                                 </module>
                                 <module name="MethodTypeParameterName">
-                                    <property name="format" value="(^[A-Z][0-9]?)$|([A-Z][a-zA-Z0-9]*[T]$)" />
+                                    <property name="format" value="(^[A-Z][0-9]?)$|([A-Z][a-zA-Z0-9]*[T]$)"/>
                                 </module>
-                                <module name="MethodParamPad" />
+                                <module name="MethodParamPad"/>
                                 <module name="OperatorWrap">
-                                    <property name="option" value="NL" />
-                                    <property name="tokens" value="BAND, BOR, BSR, BXOR, DIV, EQUAL, GE, GT, LAND, LE, LITERAL_INSTANCEOF, LOR, LT, MINUS, MOD, NOT_EQUAL, QUESTION, SL, SR, STAR " />
+                                    <property name="option" value="NL"/>
+                                    <property name="tokens" value="BAND, BOR, BSR, BXOR, DIV, EQUAL, GE, GT, LAND, LE, LITERAL_INSTANCEOF, LOR, LT, MINUS, MOD, NOT_EQUAL, QUESTION, SL, SR, STAR "/>
                                 </module>
                                 <module name="AnnotationLocation">
-                                    <property name="tokens" value="CLASS_DEF, INTERFACE_DEF, ENUM_DEF, METHOD_DEF, CTOR_DEF" />
+                                    <property name="tokens" value="CLASS_DEF, INTERFACE_DEF, ENUM_DEF, METHOD_DEF, CTOR_DEF"/>
                                 </module>
                                 <module name="AnnotationLocation">
-                                    <property name="tokens" value="VARIABLE_DEF" />
-                                    <property name="allowSamelineMultipleAnnotations" value="true" />
+                                    <property name="tokens" value="VARIABLE_DEF"/>
+                                    <property name="allowSamelineMultipleAnnotations" value="true"/>
                                 </module>
-                                <module name="NonEmptyAtclauseDescription" />
+                                <module name="NonEmptyAtclauseDescription"/>
                                 <module name="JavadocMethod">
-                                    <property name="allowUndeclaredRTE" value="true" />
-                                    <property name="allowMissingJavadoc" value="true" />
-                                    <property name="allowMissingParamTags" value="true" />
-                                    <property name="allowMissingThrowsTags" value="true" />
-                                    <property name="allowMissingReturnTag" value="true" />
-                                    <property name="allowedAnnotations" value="Override,Test,BeforeClass,AfterClass,Before,After" />
-                                    <property name="allowThrowsTagsForSubclasses" value="true" />
+                                    <property name="allowUndeclaredRTE" value="true"/>
+                                    <property name="allowMissingJavadoc" value="true"/>
+                                    <property name="allowMissingParamTags" value="true"/>
+                                    <property name="allowMissingThrowsTags" value="true"/>
+                                    <property name="allowMissingReturnTag" value="true"/>
+                                    <property name="allowedAnnotations" value="Override,Test,BeforeClass,AfterClass,Before,After"/>
+                                    <property name="allowThrowsTagsForSubclasses" value="true"/>
                                 </module>
-                                <module name="SingleLineJavadoc" />
+                                <module name="SingleLineJavadoc"/>
                             </module>
                         </module>
                     </checkstyleRules>