You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by ex...@apache.org on 2021/12/02 21:11:05 UTC

[nifi] branch main updated: NIFI-9069 Changed framework dataflow serialization to support JSON

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

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


The following commit(s) were added to refs/heads/main by this push:
     new 90b39b5  NIFI-9069 Changed framework dataflow serialization to support JSON
90b39b5 is described below

commit 90b39b593a0958122f950ed1b49e4098d2935d2a
Author: Mark Payne <ma...@hotmail.com>
AuthorDate: Tue Aug 10 14:04:13 2021 -0400

    NIFI-9069 Changed framework dataflow serialization to support JSON
    
    - Changed framework so that it serializes the dataflow into a VersionedDataflow using JSON as well as XML, and prefers the JSON representation on load, if it's available. This also results in the need for the cluster protocol to exchange its representation of the dataflow to using JSON. Rather than re-implementing all of the complex logic of Flow Fingerprinting, updated to just inherit the cluster's flow.
    - Moved logic to synchronize Process Group with Versioned Process Group into a new ProcessGroupSynchronizer class instead of having all of the logic within StandardProcessGroup
    - Reworked versioned components to use an instance id.
    - Renamed StandardFlowSynchronizer to XmlFlowSynchronizer; introduced new StandardFlowSynchronizer that delegates to the appropriate (Xml or Versioned)FlowSynchronzer
    - Updated to allow import of VersionedProcessGroup even if not all bundles are available - will now use ghost components
    - Introduced a VersionedDataflow object to hold controller-level services, reporting tasks, parameter contexts, templates, etc.
    - Allow mutable requests to be made while nodes are disconnected. Also fixed issue in AbstractPolicyBasedAuthorizer that caused ClassNotFoundException / NoClassDefFoundError if the authorizations were changed and then a node attempts to rejoin the cluster. The Authorizer was attempting to use XmlUtils, which is in nifi-security-utils and so so by madking nifi-security-utils a provided dependency of nifi-framework-api, but this doesn't work, because nifi-framework-api is loaded by a hi [...]
    - Fixed bug that occurred when importing a Process Group that has 2 parameter contexts, one inheriting from another, where neither is pre-defined in the existing flow
    - Fixed bug that was encountered when Updating a Versioned Process Group where one version had a disabled processor and the other had the processor running.
    - Increased system-tests workflow timeout to 120 minutes
    - Added additional exception handling to system tests
    
    This closes #5514
    
    Signed-off-by: David Handermann <ex...@apache.org>
---
 .github/workflows/system-tests.yml                 |   17 +-
 .../src/main/java/org/apache/nifi/flow/Bundle.java |   13 +-
 .../java/org/apache/nifi/flow/ComponentType.java   |    7 +-
 .../org/apache/nifi/flow/ConnectableComponent.java |   21 +
 .../java/org/apache/nifi/flow/ScheduledState.java  |    3 +-
 .../org/apache/nifi/flow/VersionedComponent.java   |   14 +-
 .../nifi/flow/VersionedControllerService.java      |   19 +-
 .../apache/nifi/flow/VersionedProcessGroup.java    |    1 +
 .../org/apache/nifi/flow/VersionedProcessor.java   |    2 +-
 ...lerService.java => VersionedReportingTask.java} |   62 +-
 .../nifi/flow/encryptor/StandardFlowEncryptor.java |   19 +-
 .../java/org/apache/nifi/util/NiFiProperties.java  |   20 +-
 .../src/main/asciidoc/administration-guide.adoc    |   61 +-
 .../AbstractPolicyBasedAuthorizer.java             |   24 +-
 .../authorization/FileAccessPolicyProvider.java    |   80 +-
 .../FileAccessPolicyProviderTest.java              |    8 +
 .../nifi/authorization/FileAuthorizerTest.java     |    9 +
 .../nifi-framework-cluster-protocol/pom.xml        |    7 +-
 .../nifi/cluster/protocol/StandardDataFlow.java    |   63 +-
 .../coordination/flow/PopularVoteFlowElection.java |   23 +-
 .../flow/PopularVoteFlowElectionFactoryBean.java   |   16 +-
 .../http/replication/RequestReplicator.java        |   11 +-
 .../replication/ThreadPoolRequestReplicator.java   |   41 -
 .../resources/nifi-cluster-manager-context.xml     |    1 -
 .../PopularVoteFlowElectionFactoryBeanTest.groovy  |   87 -
 .../flow/TestPopularVoteFlowElection.java          |   60 +-
 .../TestThreadPoolRequestReplicator.java           |   77 -
 .../apache/nifi/cluster/integration/Cluster.java   |   10 +-
 .../src/test/resources/conf/non-empty-flow.xml     |    1 +
 .../nifi/controller/StandardProcessorNode.java     |    5 +-
 .../nifi/controller/flow/AbstractFlowManager.java  |   26 +-
 .../reporting/AbstractReportingTaskNode.java       |   11 +-
 .../service/StandardControllerServiceProvider.java |   24 +-
 .../nifi/groups/DefaultComponentScheduler.java     |   51 +
 .../groups/ProcessGroupSynchronizationContext.java |  163 ++
 .../nifi/groups/ProcessGroupSynchronizer.java      |   12 +-
 .../apache/nifi/groups/StandardProcessGroup.java   | 1660 +---------------
 .../groups/StandardProcessGroupSynchronizer.java   | 2084 ++++++++++++++++++++
 .../mapping/InstantiatedConnectableComponent.java  |    2 +-
 .../mapping/InstantiatedVersionedComponent.java    |    4 +-
 .../mapping/InstantiatedVersionedConnection.java   |    2 +-
 .../InstantiatedVersionedControllerService.java    |    2 +-
 .../flow/mapping/InstantiatedVersionedFunnel.java  |    2 +-
 .../flow/mapping/InstantiatedVersionedLabel.java   |    2 +-
 .../flow/mapping/InstantiatedVersionedPort.java    |    2 +-
 .../mapping/InstantiatedVersionedProcessGroup.java |    2 +-
 .../mapping/InstantiatedVersionedProcessor.java    |    2 +-
 .../InstantiatedVersionedRemoteGroupPort.java      |    2 +-
 .../InstantiatedVersionedRemoteProcessGroup.java   |    2 +-
 .../flow/mapping/NiFiRegistryFlowMapper.java       |  197 +-
 .../flow/mapping/StandardComparableDataFlow.java   |   32 +
 .../apache/nifi/util/FlowDifferenceFilters.java    |  111 +-
 .../flow/mapping/TestNiFiRegistryFlowMapper.java   |  221 +++
 .../org/apache/nifi/cluster/protocol/DataFlow.java |   10 +
 .../nifi/controller/AbstractComponentNode.java     |   25 +-
 .../org/apache/nifi/controller/ProcessorNode.java  |    2 +-
 .../apache/nifi/controller/flow/FlowManager.java   |    9 +-
 .../nifi/controller/flow/VersionedDataflow.java    |  101 +
 .../flow/VersionedFlowEncodingVersion.java         |   40 +-
 .../nifi/controller/flow/VersionedRegistry.java    |   53 +-
 .../nifi/controller/flow/VersionedTemplate.java    |   28 +-
 .../nifi/groups/AbstractComponentScheduler.java    |   72 +
 .../apache/nifi/groups/BundleUpdateStrategy.java   |   16 +-
 .../ComponentIdGenerator.java}                     |   37 +-
 .../apache/nifi/groups/ComponentScheduler.java}    |   29 +-
 .../nifi/groups/GroupSynchronizationOptions.java   |  220 +++
 .../java/org/apache/nifi/groups/ProcessGroup.java  |   12 +-
 .../org/apache/nifi/groups/PropertyDecryptor.java  |   10 +-
 .../registry/flow/mapping/ComponentIdLookup.java   |   55 +
 .../registry/flow/mapping/FlowMappingOptions.java  |  197 ++
 .../flow/mapping/SensitiveValueEncryptor.java      |    8 +-
 .../mapping/VersionedComponentStateLookup.java     |  108 +
 .../org/apache/nifi/controller/FlowController.java |   19 +-
 .../nifi/controller/StandardFlowService.java       |   35 +-
 ...wSynchronizer.java => XmlFlowSynchronizer.java} |   27 +-
 .../inheritance/BundleCompatibilityCheck.java      |  100 +-
 .../inheritance/ConnectionMissingCheck.java        |   72 +-
 .../repository/WriteAheadFlowFileRepository.java   |    4 +-
 .../serialization/AffectedComponentSet.java        |  645 ++++++
 .../controller/serialization/FlowSynchronizer.java |    3 +-
 .../serialization/ScheduledStateLookup.java        |    3 +-
 .../serialization/StandardFlowSynchronizer.java    |   65 +
 .../serialization/VersionedDataflowMapper.java     |  221 +++
 .../serialization/VersionedFlowSerializer.java     |   70 +
 .../serialization/VersionedFlowSynchronizer.java   |  978 +++++++++
 .../FlowConfigurationArchiveManager.java           |   44 +-
 .../nifi/persistence/FlowConfigurationDAO.java     |   13 +-
 .../persistence/StandardFlowConfigurationDAO.java  |  229 +++
 .../StandardXMLFlowConfigurationDAO.java           |  190 --
 .../java/org/apache/nifi/util/BundleUtils.java     |   52 +-
 .../main/java/org/apache/nifi/util/FlowParser.java |  161 +-
 .../apache/nifi/controller/TestFlowController.java |   72 +-
 .../controller/service/mock/MockProcessGroup.java  |    6 +
 .../nifi/integration/FrameworkIntegrationTest.java |   15 +-
 .../nifi/integration/versioned/ImportFlowIT.java   |   25 +-
 .../TestFlowConfigurationArchiveManager.java       |   48 +-
 .../flow/mapping/NiFiRegistryFlowMapperTest.java   |    1 -
 .../org/apache/nifi/headless/FlowEnricher.java     |  264 ---
 .../apache/nifi/headless/HeadlessNiFiServer.java   |    2 -
 .../nifi/properties/NiFiPropertiesLoader.java      |   13 +-
 .../nifi-framework/nifi-resources/pom.xml          |    1 +
 .../src/main/resources/conf/nifi.properties        |    1 +
 .../nifi/registry/flow/FlowRegistryUtils.java      |    3 +-
 .../apache/nifi/web/StandardNiFiServiceFacade.java |   79 +-
 .../apache/nifi/web/api/FlowUpdateResource.java    |   41 +-
 ...connectedNodeMutableRequestExceptionMapper.java |   49 -
 .../org/apache/nifi/web/api/dto/DtoFactory.java    |   56 +-
 .../web/dao/impl/StandardParameterContextDAO.java  |    8 +-
 .../nifi/web/StandardNiFiServiceFacadeSpec.groovy  |    8 +-
 .../registry/flow/VersionedParameterContext.java   |   22 +-
 .../registry/flow/diff/ComparableDataFlow.java     |   11 +
 .../diff/ConciseEvolvingDifferenceDescriptor.java  |   10 +-
 .../nifi/registry/flow/diff/DifferenceType.java    |   45 +
 .../flow/diff/EvolvingDifferenceDescriptor.java    |    6 +-
 .../flow/diff/StandardComparableDataFlow.java      |   32 +
 .../registry/flow/diff/StandardFlowComparator.java |  133 +-
 .../flow/diff/StaticDifferenceDescriptor.java      |   14 +-
 .../nifi/registry/service/RegistryService.java     |    3 +-
 .../nifi/tests/system/AggregateNiFiInstance.java   |    9 +
 .../tests/system/ExceptionalBooleanSupplier.java   |    8 +-
 .../apache/nifi/tests/system/NiFiClientUtil.java   |  200 +-
 .../org/apache/nifi/tests/system/NiFiInstance.java |    8 +
 .../org/apache/nifi/tests/system/NiFiSystemIT.java |   82 +-
 .../SpawnedStandaloneNiFiInstanceFactory.java      |   22 +-
 .../system/clustering/FlowSynchronizationIT.java   |  896 +++++++++
 .../clustering/JoinClusterWithDifferentFlow.java   |    5 +
 ... JoinClusterWithMissingConnectionWithData.java} |    2 +-
 .../system/restart/FlowFileRestorationIT.java      |    7 +-
 .../resources/conf/clustered/node2/logback.xml     |    4 +-
 .../cli/impl/client/nifi/ConnectionClient.java     |    6 +
 .../cli/impl/client/nifi/ProcessorClient.java      |    4 +
 .../cli/impl/client/nifi/ReportingTasksClient.java |    2 +
 .../impl/client/nifi/impl/CRUDJerseyClient.java    |    6 +-
 .../client/nifi/impl/JerseyConnectionClient.java   |   24 +-
 .../nifi/impl/JerseyControllerServicesClient.java  |    6 +-
 .../client/nifi/impl/JerseyProcessorClient.java    |   41 +-
 .../nifi/impl/JerseyReportingTasksClient.java      |   30 +
 137 files changed, 8551 insertions(+), 3140 deletions(-)

diff --git a/.github/workflows/system-tests.yml b/.github/workflows/system-tests.yml
index 38de21f..3ffa504 100644
--- a/.github/workflows/system-tests.yml
+++ b/.github/workflows/system-tests.yml
@@ -53,7 +53,7 @@ env:
 
 jobs:
   ubuntu:
-    timeout-minutes: 60
+    timeout-minutes: 120
     runs-on: ubuntu-latest
     name: Ubuntu Java 11
     steps:
@@ -88,8 +88,15 @@ jobs:
           ${{ env.MAVEN_COMMAND }}
           ${{ env.MAVEN_RUN_ARGUMENTS }}
           ${{ env.MAVEN_PROJECTS }}
+      - name: Upload Troubleshooting Logs
+        uses: actions/upload-artifact@v2
+        with:
+          name: ubuntu-latest-troubleshooting-logs
+          path: "**/target/troubleshooting/"
+          retention-days: 7
+
   macos:
-    timeout-minutes: 60
+    timeout-minutes: 120
     runs-on: macos-latest
     name: MacOS Java 8
     steps:
@@ -124,3 +131,9 @@ jobs:
           ${{ env.MAVEN_COMMAND }}
           ${{ env.MAVEN_RUN_ARGUMENTS }}
           ${{ env.MAVEN_PROJECTS }}
+      - name: Upload Troubleshooting Logs
+        uses: actions/upload-artifact@v2
+        with:
+          name: macos-latest-troubleshooting-logs
+          path: "**/target/troubleshooting/"
+          retention-days: 7
diff --git a/nifi-api/src/main/java/org/apache/nifi/flow/Bundle.java b/nifi-api/src/main/java/org/apache/nifi/flow/Bundle.java
index ef68f4c..019b052 100644
--- a/nifi-api/src/main/java/org/apache/nifi/flow/Bundle.java
+++ b/nifi-api/src/main/java/org/apache/nifi/flow/Bundle.java
@@ -17,10 +17,10 @@
 
 package org.apache.nifi.flow;
 
-import java.util.Objects;
-
 import io.swagger.annotations.ApiModelProperty;
 
+import java.util.Objects;
+
 public class Bundle {
     private String group;
     private String artifact;
@@ -80,4 +80,13 @@ public class Bundle {
     public int hashCode() {
         return Objects.hash(group, artifact, version);
     }
+
+    @Override
+    public String toString() {
+        return "Bundle[" +
+            "group='" + group + "'" +
+            ", artifact='" + artifact + "'" +
+            ", version='" + version + "'" +
+            "]";
+    }
 }
diff --git a/nifi-api/src/main/java/org/apache/nifi/flow/ComponentType.java b/nifi-api/src/main/java/org/apache/nifi/flow/ComponentType.java
index 954f164..56f9fb7 100644
--- a/nifi-api/src/main/java/org/apache/nifi/flow/ComponentType.java
+++ b/nifi-api/src/main/java/org/apache/nifi/flow/ComponentType.java
@@ -29,12 +29,15 @@ public enum ComponentType {
     REMOTE_OUTPUT_PORT("Remote Output Port"),
     FUNNEL("Funnel"),
     LABEL("Label"),
-    CONTROLLER_SERVICE("Controller Service");
+    CONTROLLER_SERVICE("Controller Service"),
+    REPORTING_TASK("Reporting Task"),
+    PARAMETER_CONTEXT("Parameter Context"),
+    TEMPLATE("Template");
 
 
     private final String typeName;
 
-    private ComponentType(final String typeName) {
+    ComponentType(final String typeName) {
         this.typeName = typeName;
     }
 
diff --git a/nifi-api/src/main/java/org/apache/nifi/flow/ConnectableComponent.java b/nifi-api/src/main/java/org/apache/nifi/flow/ConnectableComponent.java
index db622ce..7926ed2 100644
--- a/nifi-api/src/main/java/org/apache/nifi/flow/ConnectableComponent.java
+++ b/nifi-api/src/main/java/org/apache/nifi/flow/ConnectableComponent.java
@@ -23,6 +23,7 @@ import java.util.Objects;
 
 public class ConnectableComponent {
     private String id;
+    private String instanceId;
     private ConnectableComponentType type;
     private String groupId;
     private String name;
@@ -37,6 +38,15 @@ public class ConnectableComponent {
         this.id = id;
     }
 
+    @ApiModelProperty("The instance ID of an existing component that is described by this VersionedComponent, or null if this is not mapped to an instantiated component")
+    public String getInstanceIdentifier() {
+        return instanceId;
+    }
+
+    public void setInstanceIdentifier(String instanceIdentifier) {
+        this.instanceId = instanceIdentifier;
+    }
+
     @ApiModelProperty(value = "The type of component the connectable is.", required = true)
     public ConnectableComponentType getType() {
         return type;
@@ -92,4 +102,15 @@ public class ConnectableComponent {
         final ConnectableComponent other = (ConnectableComponent) obj;
         return Objects.equals(id, other.id);
     }
+
+    @Override
+    public String toString() {
+        return "ConnectableComponent{" +
+            "id='" + id + '\'' +
+            ", instanceId='" + instanceId + '\'' +
+            ", type=" + type +
+            ", groupId='" + groupId + '\'' +
+            ", name='" + name + '\'' +
+            '}';
+    }
 }
diff --git a/nifi-api/src/main/java/org/apache/nifi/flow/ScheduledState.java b/nifi-api/src/main/java/org/apache/nifi/flow/ScheduledState.java
index aaba9ae..d53ede9 100644
--- a/nifi-api/src/main/java/org/apache/nifi/flow/ScheduledState.java
+++ b/nifi-api/src/main/java/org/apache/nifi/flow/ScheduledState.java
@@ -18,5 +18,6 @@ package org.apache.nifi.flow;
 
 public enum ScheduledState {
     ENABLED,
-    DISABLED;
+    DISABLED,
+    RUNNING;
 }
diff --git a/nifi-api/src/main/java/org/apache/nifi/flow/VersionedComponent.java b/nifi-api/src/main/java/org/apache/nifi/flow/VersionedComponent.java
index d314bd5..639d6fb 100644
--- a/nifi-api/src/main/java/org/apache/nifi/flow/VersionedComponent.java
+++ b/nifi-api/src/main/java/org/apache/nifi/flow/VersionedComponent.java
@@ -17,14 +17,15 @@
 
 package org.apache.nifi.flow;
 
-import java.util.Objects;
-
 import io.swagger.annotations.ApiModelProperty;
 
+import java.util.Objects;
+
 
 public abstract class VersionedComponent {
 
     private String identifier;
+    private String instanceIdentifier;
     private String groupId;
     private String name;
     private String comments;
@@ -39,6 +40,15 @@ public abstract class VersionedComponent {
         this.identifier = identifier;
     }
 
+    @ApiModelProperty("The instance ID of an existing component that is described by this VersionedComponent, or null if this is not mapped to an instantiated component")
+    public String getInstanceIdentifier() {
+        return instanceIdentifier;
+    }
+
+    public void setInstanceIdentifier(String instanceIdentifier) {
+        this.instanceIdentifier = instanceIdentifier;
+    }
+
     @ApiModelProperty("The ID of the Process Group that this component belongs to")
     public String getGroupIdentifier() {
         return groupId;
diff --git a/nifi-api/src/main/java/org/apache/nifi/flow/VersionedControllerService.java b/nifi-api/src/main/java/org/apache/nifi/flow/VersionedControllerService.java
index 232339a..bd6d8dc 100644
--- a/nifi-api/src/main/java/org/apache/nifi/flow/VersionedControllerService.java
+++ b/nifi-api/src/main/java/org/apache/nifi/flow/VersionedControllerService.java
@@ -17,11 +17,11 @@
 
 package org.apache.nifi.flow;
 
+import io.swagger.annotations.ApiModelProperty;
+
 import java.util.List;
 import java.util.Map;
 
-import io.swagger.annotations.ApiModelProperty;
-
 public class VersionedControllerService extends VersionedComponent
         implements VersionedConfigurableComponent, VersionedExtensionComponent {
 
@@ -32,7 +32,7 @@ public class VersionedControllerService extends VersionedComponent
     private Map<String, String> properties;
     private Map<String, VersionedPropertyDescriptor> propertyDescriptors;
     private String annotationData;
-
+    private ScheduledState scheduledState;
 
     @Override
     @ApiModelProperty(value = "The type of the controller service.")
@@ -46,7 +46,7 @@ public class VersionedControllerService extends VersionedComponent
     }
 
     @Override
-    @ApiModelProperty(value = "The details of the artifact that bundled this processor type.")
+    @ApiModelProperty(value = "The details of the artifact that bundled this controller service type.")
     public Bundle getBundle() {
         return bundle;
     }
@@ -77,7 +77,7 @@ public class VersionedControllerService extends VersionedComponent
     }
 
     @Override
-    @ApiModelProperty("The property descriptors for the processor.")
+    @ApiModelProperty("The property descriptors for the controller service.")
     public Map<String, VersionedPropertyDescriptor> getPropertyDescriptors() {
         return propertyDescriptors;
     }
@@ -100,4 +100,13 @@ public class VersionedControllerService extends VersionedComponent
     public ComponentType getComponentType() {
         return ComponentType.CONTROLLER_SERVICE;
     }
+
+    @ApiModelProperty("The ScheduledState denoting whether the Controller Service is ENABLED or DISABLED")
+    public ScheduledState getScheduledState() {
+        return scheduledState;
+    }
+
+    public void setScheduledState(final ScheduledState scheduledState) {
+        this.scheduledState = scheduledState;
+    }
 }
diff --git a/nifi-api/src/main/java/org/apache/nifi/flow/VersionedProcessGroup.java b/nifi-api/src/main/java/org/apache/nifi/flow/VersionedProcessGroup.java
index 4d2f527..ea4edf2 100644
--- a/nifi-api/src/main/java/org/apache/nifi/flow/VersionedProcessGroup.java
+++ b/nifi-api/src/main/java/org/apache/nifi/flow/VersionedProcessGroup.java
@@ -47,6 +47,7 @@ public class VersionedProcessGroup extends VersionedComponent {
     private Long defaultBackPressureObjectThreshold;
     private String defaultBackPressureDataSizeThreshold;
 
+
     @ApiModelProperty("The child Process Groups")
     public Set<VersionedProcessGroup> getProcessGroups() {
         return processGroups;
diff --git a/nifi-api/src/main/java/org/apache/nifi/flow/VersionedProcessor.java b/nifi-api/src/main/java/org/apache/nifi/flow/VersionedProcessor.java
index e519f53..51c42de 100644
--- a/nifi-api/src/main/java/org/apache/nifi/flow/VersionedProcessor.java
+++ b/nifi-api/src/main/java/org/apache/nifi/flow/VersionedProcessor.java
@@ -53,7 +53,7 @@ public class VersionedProcessor extends VersionedComponent
         this.schedulingPeriod = setSchedulingPeriod;
     }
 
-    @ApiModelProperty("Indcates whether the prcessor should be scheduled to run in event or timer driven mode.")
+    @ApiModelProperty("Indicates whether the processor should be scheduled to run in event or timer driven mode.")
     public String getSchedulingStrategy() {
         return schedulingStrategy;
     }
diff --git a/nifi-api/src/main/java/org/apache/nifi/flow/VersionedControllerService.java b/nifi-api/src/main/java/org/apache/nifi/flow/VersionedReportingTask.java
similarity index 57%
copy from nifi-api/src/main/java/org/apache/nifi/flow/VersionedControllerService.java
copy to nifi-api/src/main/java/org/apache/nifi/flow/VersionedReportingTask.java
index 232339a..38ca294 100644
--- a/nifi-api/src/main/java/org/apache/nifi/flow/VersionedControllerService.java
+++ b/nifi-api/src/main/java/org/apache/nifi/flow/VersionedReportingTask.java
@@ -17,36 +17,34 @@
 
 package org.apache.nifi.flow;
 
-import java.util.List;
-import java.util.Map;
-
 import io.swagger.annotations.ApiModelProperty;
 
-public class VersionedControllerService extends VersionedComponent
-        implements VersionedConfigurableComponent, VersionedExtensionComponent {
+import java.util.Map;
 
+public class VersionedReportingTask extends VersionedComponent implements VersionedConfigurableComponent, VersionedExtensionComponent {
     private String type;
     private Bundle bundle;
-    private List<ControllerServiceAPI> controllerServiceApis;
-
     private Map<String, String> properties;
     private Map<String, VersionedPropertyDescriptor> propertyDescriptors;
     private String annotationData;
+    private ScheduledState scheduledState;
+    private String schedulingPeriod;
+    private String schedulingStrategy;
 
 
     @Override
-    @ApiModelProperty(value = "The type of the controller service.")
+    @ApiModelProperty(value = "The type of the reporting task.")
     public String getType() {
         return type;
     }
 
     @Override
-    public void setType(String type) {
+    public void setType(final String type) {
         this.type = type;
     }
 
     @Override
-    @ApiModelProperty(value = "The details of the artifact that bundled this processor type.")
+    @ApiModelProperty(value = "The details of the artifact that bundled this reporting task type.")
     public Bundle getBundle() {
         return bundle;
     }
@@ -56,17 +54,8 @@ public class VersionedControllerService extends VersionedComponent
         this.bundle = bundle;
     }
 
-    @ApiModelProperty(value = "Lists the APIs this Controller Service implements.")
-    public List<ControllerServiceAPI> getControllerServiceApis() {
-        return controllerServiceApis;
-    }
-
-    public void setControllerServiceApis(List<ControllerServiceAPI> controllerServiceApis) {
-        this.controllerServiceApis = controllerServiceApis;
-    }
-
     @Override
-    @ApiModelProperty(value = "The properties of the controller service.")
+    @ApiModelProperty(value = "The properties of the reporting task.")
     public Map<String, String> getProperties() {
         return properties;
     }
@@ -77,7 +66,7 @@ public class VersionedControllerService extends VersionedComponent
     }
 
     @Override
-    @ApiModelProperty("The property descriptors for the processor.")
+    @ApiModelProperty("The property descriptors for the reporting task.")
     public Map<String, VersionedPropertyDescriptor> getPropertyDescriptors() {
         return propertyDescriptors;
     }
@@ -87,7 +76,7 @@ public class VersionedControllerService extends VersionedComponent
         this.propertyDescriptors = propertyDescriptors;
     }
 
-    @ApiModelProperty(value = "The annotation for the controller service. This is how the custom UI relays configuration to the controller service.")
+    @ApiModelProperty(value = "The annotation for the reporting task. This is how the custom UI relays configuration to the reporting task.")
     public String getAnnotationData() {
         return annotationData;
     }
@@ -96,8 +85,35 @@ public class VersionedControllerService extends VersionedComponent
         this.annotationData = annotationData;
     }
 
+    @ApiModelProperty("The frequency with which to schedule the reporting task. The format of the value will depend on the value of schedulingStrategy.")
+    public String getSchedulingPeriod() {
+        return schedulingPeriod;
+    }
+
+    public void setSchedulingPeriod(String setSchedulingPeriod) {
+        this.schedulingPeriod = setSchedulingPeriod;
+    }
+
+    @ApiModelProperty("Indicates scheduling strategy that should dictate how the reporting task is triggered.")
+    public String getSchedulingStrategy() {
+        return schedulingStrategy;
+    }
+
+    public void setSchedulingStrategy(String schedulingStrategy) {
+        this.schedulingStrategy = schedulingStrategy;
+    }
+
     @Override
     public ComponentType getComponentType() {
-        return ComponentType.CONTROLLER_SERVICE;
+        return ComponentType.REPORTING_TASK;
+    }
+
+    @ApiModelProperty("Indicates the scheduled state for the Reporting Task")
+    public ScheduledState getScheduledState() {
+        return scheduledState;
+    }
+
+    public void setScheduledState(final ScheduledState scheduledState) {
+        this.scheduledState = scheduledState;
     }
 }
diff --git a/nifi-commons/nifi-flow-encryptor/src/main/java/org/apache/nifi/flow/encryptor/StandardFlowEncryptor.java b/nifi-commons/nifi-flow-encryptor/src/main/java/org/apache/nifi/flow/encryptor/StandardFlowEncryptor.java
index b29613b..43e0901 100644
--- a/nifi-commons/nifi-flow-encryptor/src/main/java/org/apache/nifi/flow/encryptor/StandardFlowEncryptor.java
+++ b/nifi-commons/nifi-flow-encryptor/src/main/java/org/apache/nifi/flow/encryptor/StandardFlowEncryptor.java
@@ -53,13 +53,24 @@ public class StandardFlowEncryptor implements FlowEncryptor {
             try (final BufferedReader reader = new BufferedReader(new InputStreamReader(inputStream))) {
                 reader.lines().forEach(line -> {
                     final Matcher matcher = ENCRYPTED_PATTERN.matcher(line);
-                    if (matcher.find()) {
+
+                    final StringBuffer sb = new StringBuffer();
+                    boolean matched = false;
+                    while (matcher.find()) {
                         final String outputEncrypted = getOutputEncrypted(matcher.group(FIRST_GROUP), inputEncryptor, outputEncryptor);
-                        final String outputLine = matcher.replaceFirst(outputEncrypted);
-                        writer.println(outputLine);
+                        matcher.appendReplacement(sb, outputEncrypted);
+                        matched = true;
+                    }
+
+                    final String outputLine;
+                    if (matched) {
+                        matcher.appendTail(sb);
+                        outputLine = sb.toString();
                     } else {
-                        writer.println(line);
+                        outputLine = line;
                     }
+
+                    writer.println(outputLine);
                 });
             }
         } catch (final IOException e) {
diff --git a/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java b/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java
index 9146249..b1524f6 100644
--- a/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java
+++ b/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java
@@ -55,6 +55,7 @@ public class NiFiProperties extends ApplicationProperties {
     // core properties
     public static final String PROPERTIES_FILE_PATH = "nifi.properties.file.path";
     public static final String FLOW_CONFIGURATION_FILE = "nifi.flow.configuration.file";
+    public static final String FLOW_CONFIGURATION_JSON_FILE = "nifi.flow.configuration.json.file";
     public static final String FLOW_CONFIGURATION_ARCHIVE_ENABLED = "nifi.flow.configuration.archive.enabled";
     public static final String FLOW_CONFIGURATION_ARCHIVE_DIR = "nifi.flow.configuration.archive.dir";
     public static final String FLOW_CONFIGURATION_ARCHIVE_MAX_TIME = "nifi.flow.configuration.archive.max.time";
@@ -420,7 +421,7 @@ public class NiFiProperties extends ApplicationProperties {
     public static final String DEFAULT_COMPONENT_STATUS_REPOSITORY_PERSIST_LOCATION = "./status_repository";
 
     public NiFiProperties() {
-        this(Collections.EMPTY_MAP);
+        this(Collections.emptyMap());
     }
 
     public NiFiProperties(final Map<String, String> props) {
@@ -440,6 +441,21 @@ public class NiFiProperties extends ApplicationProperties {
         }
     }
 
+    public File getFlowConfigurationJsonFile() {
+        final String jsonFilename = getProperty(FLOW_CONFIGURATION_JSON_FILE);
+        if (jsonFilename != null) {
+            return new File(jsonFilename);
+        }
+
+        final File xmlFile = getFlowConfigurationFile();
+        final String xmlFilename = xmlFile.getName();
+        if (xmlFilename.contains(".xml")) {
+            return new File(xmlFile.getParentFile(), xmlFilename.replace(".xml", ".json"));
+        }
+
+        return new File(xmlFile.getParentFile(), xmlFilename.replace(".gz", "") + ".json.gz");
+    }
+
     public File getFlowConfigurationFileDir() {
         try {
             return getFlowConfigurationFile().getParentFile();
@@ -2022,7 +2038,7 @@ public class NiFiProperties extends ApplicationProperties {
      * @return NiFiProperties
      */
     public static NiFiProperties createBasicNiFiProperties(final String propertiesFilePath, final Map<String, String> additionalProperties) {
-        final Map<String, String> addProps = (additionalProperties == null) ? Collections.EMPTY_MAP : additionalProperties;
+        final Map<String, String> addProps = (additionalProperties == null) ? Collections.emptyMap() : additionalProperties;
         final Properties properties = new Properties();
         addProps.forEach(properties::put);
 
diff --git a/nifi-docs/src/main/asciidoc/administration-guide.adoc b/nifi-docs/src/main/asciidoc/administration-guide.adoc
index d414e52..c749a0f 100644
--- a/nifi-docs/src/main/asciidoc/administration-guide.adoc
+++ b/nifi-docs/src/main/asciidoc/administration-guide.adoc
@@ -70,7 +70,7 @@ When NiFi first starts up, the following files and directories are created:
 * `provenance_repository`
 * `work` directory
 * `logs` directory
-* Within the `conf` directory, the _flow.xml.gz_ file is created
+* Within the `conf` directory, the _flow.json.gz_ file is created
 
 NOTE: For security purposes, when no security configuration is provided NiFi will now bind to 127.0.0.1 by default and the UI will only be accessible through this loopback interface. HTTPS properties should be configured to access NiFi from other interfaces. See the <<security_configuration,Security Configuration>> for guidance on how to do this.
 
@@ -732,7 +732,11 @@ If you are setting up a secured NiFi instance for the first time, you must manua
 
 After you have edited and saved the _authorizers.xml_ file, restart NiFi.  The “Initial Admin Identity” user and administrative policies are added to the _users.xml_ and _authorizations.xml_ files during restart. Once NiFi starts, the “Initial Admin Identity” user is able to access the UI and begin managing users, groups, and policies.
 
-NOTE: For a brand new secure flow, providing the "Initial Admin Identity" gives that user access to get into the UI and to manage users, groups and policies.  But if that user wants to start modifying the flow, they need to grant themselves policies for the root process group. The system is unable to do this automatically because in a new flow the UUID of the root process group is not permanent until the _flow.xml.gz_ is generated.  If the NiFi instance is an upgrade from an existing _fl [...]
+NOTE: For a brand new secure flow, providing the "Initial Admin Identity" gives that user access to get into the UI and to manage users, groups and policies.  But if that user wants to start
+modifying the flow, they need to grant themselves policies for the root process group. The system is unable to do this automatically because in a new flow the UUID of the root process group is not
+permanent until the _flow.json.gz_ is generated.  If the NiFi instance is an upgrade from an existing _flow.json.gz_ or a 1.x instance going from unsecure to secure, then the "Initial Admin Identity"
+user
+is automatically given the privileges to modify the flow.
 
 Some common use cases are described below.
 
@@ -1128,7 +1132,7 @@ Here is an example entry:
 
 After you have edited and saved the _authorizers.xml_ file, restart NiFi. Users and roles from the _authorized-users.xml_ file are converted and added as identities and policies in the _users.xml_ and _authorizations.xml_ files.  Once the application starts, users who previously had a legacy Administrator role can access the UI and begin managing users, groups, and policies.
 
-The following tables summarize the global and component policies assigned to each legacy role if the NiFi instance has an existing _flow.xml.gz_:
+The following tables summarize the global and component policies assigned to each legacy role if the NiFi instance has an existing _flow.json.gz_:
 
 ===== Global Access Policies
 [cols=">s,^s,^s,^s,^s,^s,^s", options="header"]
@@ -2164,9 +2168,9 @@ Any node whose dataflow, users, groups, and policies conflict with those elected
 resources with those from the cluster. How the backup is performed depends on the configured Access Policy Provider and User Group Provider.
 For file-based access policy providers, the backup will be written to the same directory as the existing file (e.g., $NIFI_HOME/conf) and bear the same
 name but with a suffix of "." and a timestamp. For example, if the flow itself conflicts with the cluster's flow at 12:05:03 on January 1, 2020,
-the node's `flow.xml.gz` file will be copied to `flow.xml.gz.2020-01-01-12-05-03` and the cluster's flow will then be written to `flow.xml.gz`.
+the node's `flow.json.gz` file will be copied to `flow.json.gz.2020-01-01-12-05-03` and the cluster's flow will then be written to `flow.json.gz`.
 Similarly, this will happen for the `users.xml` and `authorizations.xml` file. This is done so that the flow can be manually reverted if necessary
-by renaming the backup file back to `flow.xml.gz`, for example.
+by renaming the backup file back to `flow.json.gz`, for example.
 
 It is important to note that before inheriting the elected flow, NiFi will first read through the FlowFile repository and any swap files to determine which
 queues in the dataflow currently hold data. If there exists any queue in the dataflow that contains a FlowFile, that queue must also exist in the elected
@@ -2175,7 +2179,7 @@ log errors to that effect and will fail to startup. This ensures that even if th
 restarting the node will not result in data loss.
 
 Election is performed according to the "popular vote" with the caveat that the winner will never be an "empty flow" unless all flows are empty. This
-allows an administrator to remove a node's _flow.xml.gz_ file and restart the node, knowing that the node's flow will
+allows an administrator to remove a node's _flow.json.gz_ file and restart the node, knowing that the node's flow will
 not be voted to be the "correct" flow unless no other flow is found. If there are two non-empty flows that receive the same number of votes, one of those
 flows will be chosen. The methodology used to determine which of those flows is undefined and may change at any time without notice.
 
@@ -3016,16 +3020,20 @@ The first section of the _nifi.properties_ file is for the Core Properties. Thes
 
 |===
 |*Property*|*Description*
-|`nifi.flow.configuration.file`*|The location of the flow configuration file (i.e., the file that contains what is currently displayed on the NiFi graph). The default value is `./conf/flow.xml.gz`.
+|`nifi.flow.configuration.file`*|The location of the XML-based flow configuration file. The default value is `./conf/flow.xml.gz`. This is a legacy property. Older versions of NiFi used an
+XML-formatted file to store the flow configuration. However, newer versions use a JSON representation. In order to maintain backward compatibility of flows and still load flows developed using
+older versions of NiFi, upon startup, NiFi will use the `nifi.flow.configuration.json.file` first. If the file exists, it will be used. However, if it does not exist, NiFi will fall back to this
+property to determine the XML version of the file and use it.
+|`nifi.flow.configuration.json.file`*|The location of the flow configuration file (i.e., the file that contains what is currently displayed on the NiFi graph). The default value is `./conf/flow.json.gz`.
 |`nifi.flow.configuration.archive.enabled`*|Specifies whether NiFi creates a backup copy of the flow automatically when the flow is updated. The default value is `true`.
-|`nifi.flow.configuration.archive.dir`*|The location of the archive directory where backup copies of the _flow.xml_ are saved. The default value is `./conf/archive`. NiFi removes old archive files to limit disk usage based on archived file lifespan, total size, and number of files, as specified with `nifi.flow.configuration.archive.max.time`, `max.storage` and `max.count` properties respectively. If none of these limitation for archiving is specified, NiFi uses default conditions, that i [...]
-This cleanup mechanism takes into account only automatically created archived _flow.xml_ files. If there are other files or directories in this archive directory, NiFi will ignore them. Automatically created archives have filename with ISO 8601 format timestamp prefix followed by `<original-filename>`. That is `<year><month><day>T<hour><minute><second>+<timezone offset>_<original filename>`. For example, `20160706T160719+0900_flow.xml.gz`. NiFi checks filenames when it cleans archive dir [...]
-|`nifi.flow.configuration.archive.max.time`*|The lifespan of archived _flow.xml_ files. NiFi will delete expired archive files when it updates _flow.xml_ if this property is specified. Expiration is determined based on current system time and the last modified timestamp of an archived _flow.xml_. If no archive limitation is specified in _nifi.properties_, NiFi removes archives older than `30 days`.
-|`nifi.flow.configuration.archive.max.storage`*|The total data size allowed for the archived _flow.xml_ files. NiFi will delete the oldest archive files until the total archived file size becomes less than this configuration value, if this property is specified. If no archive limitation is specified in _nifi.properties_, NiFi uses `500 MB` for this.
+|`nifi.flow.configuration.archive.dir`*|The location of the archive directory where backup copies of the _flow.json_ are saved. The default value is `./conf/archive`. NiFi removes old archive files to limit disk usage based on archived file lifespan, total size, and number of files, as specified with `nifi.flow.configuration.archive.max.time`, `max.storage` and `max.count` properties respectively. If none of these limitation for archiving is specified, NiFi uses default conditions, that  [...]
+This cleanup mechanism takes into account only automatically created archived _flow.json_ files. If there are other files or directories in this archive directory, NiFi will ignore them. Automatically created archives have filename with ISO 8601 format timestamp prefix followed by `<original-filename>`. That is `<year><month><day>T<hour><minute><second>+<timezone offset>_<original filename>`. For example, `20160706T160719+0900_flow.json.gz`. NiFi checks filenames when it cleans archive d [...]
+|`nifi.flow.configuration.archive.max.time`*|The lifespan of archived _flow.json_ files. NiFi will delete expired archive files when it updates _flow.json_ if this property is specified. Expiration is determined based on current system time and the last modified timestamp of an archived _flow.json_. If no archive limitation is specified in _nifi.properties_, NiFi removes archives older than `30 days`.
+|`nifi.flow.configuration.archive.max.storage`*|The total data size allowed for the archived _flow.json_ files. NiFi will delete the oldest archive files until the total archived file size becomes less than this configuration value, if this property is specified. If no archive limitation is specified in _nifi.properties_, NiFi uses `500 MB` for this.
 |`nifi.flow.configuration.archive.max.count`*|The number of archive files allowed. NiFi will delete the oldest archive files so that only N latest archives can be kept, if this property is specified.
 |`nifi.flowcontroller.autoResumeState`|Indicates whether -upon restart- the components on the NiFi graph should return to their last state. The default value is `true`.
 |`nifi.flowcontroller.graceful.shutdown.period`|Indicates the shutdown period. The default value is `10 secs`.
-|`nifi.flowservice.writedelay.interval`|When many changes are made to the _flow.xml_, this property specifies how long to wait before writing out the changes, so as to batch the changes into a single write. The default value is `500 ms`.
+|`nifi.flowservice.writedelay.interval`|When many changes are made to the _flow.json_, this property specifies how long to wait before writing out the changes, so as to batch the changes into a single write. The default value is `500 ms`.
 |`nifi.administrative.yield.duration`|If a component allows an unexpected exception to escape, it is considered a bug. As a result, the framework will pause (or administratively yield) the component for this amount of time. This is done so that the component does not use up massive amounts of system resources, since it is known to have problems in the existing state. The default value is `30 secs`.
 |`nifi.bored.yield.duration`|When a component has no work to do (i.e., is "bored"), this is the amount of time it will wait before checking to see if it has new data to work on. This way, it does not use up CPU resources by checking for new work too often. When setting this property, be aware that it could add extra latency for components that do not constantly have work to do, as once they go into this "bored" state, they will wait this amount of time before checking for more work. The  [...]
 |`nifi.queue.backpressure.count`|When drawing a new connection between two components, this is the default value for that connection's back pressure object threshold. The default is `10000` and the value must be an integer.
@@ -3033,7 +3041,7 @@ This cleanup mechanism takes into account only automatically created archived _f
 |`nifi.authorizer.configuration.file`*|This is the location of the file that specifies how authorizers are defined.  The default value is `./conf/authorizers.xml`.
 |`nifi.login.identity.provider.configuration.file`*|This is the location of the file that specifies how username/password authentication is performed. This file is
 only considered if `nifi.security.user.login.identity.provider` is configured with a provider identifier. The default value is `./conf/login-identity-providers.xml`.
-|`nifi.templates.directory`*|This is the location of the directory where flow templates are saved (for backward compatibility only).  Templates are stored in the _flow.xml.gz_ starting with NiFi 1.0. The template directory can be used to (bulk) import templates into the _flow.xml.gz_ automatically on NiFi startup.  The default value is `./conf/templates`.
+|`nifi.templates.directory`*|This is the location of the directory where flow templates are saved (for backward compatibility only).  Templates are stored in the _flow.json.gz_ starting with NiFi 1.0. The template directory can be used to (bulk) import templates into the _flow.json.gz_ automatically on NiFi startup.  The default value is `./conf/templates`.
 |`nifi.ui.banner.text`|This is banner text that may be configured to display at the top of the User Interface. It is blank by default.
 |`nifi.ui.autorefresh.interval`|The interval at which the User Interface auto-refreshes. The default value is `30 secs`.
 |`nifi.nar.library.directory`|The location of the nar library. The default value is `./lib` and probably should be left as is. 
@@ -4212,9 +4220,9 @@ If you are using the `file-provider` authorizer, ensure that you copy the _users
 Configuration best practices recommend creating a separate location outside of the NiFi base directory for storing such configuration files, for example: `/opt/nifi/configuration-resources/`. If you are storing these files in a separate directory, you do not need to move them. Instead, ensure that the new NiFi is pointing to the same files.
 |_bootstrap-notification-services.xml_       | Use the existing NiFi _bootstrap-notification-services.xml_  file to update properties in the new NiFi.
 |_bootstrap.conf_                            | Use the existing NiFi _bootstrap.conf_ file to update properties in the new NiFi.
-|_flow.xml.gz_                               | If you retained the default location for storing flows (`<installation-directory>/conf/`), copy _flow.xml.gz_ from the existing to the new NiFi base install `conf` directory. If you stored flows to an external location via _nifi.properties_, update the property `nifi.flow.configuration.file` to point there.
+|_flow.json.gz_                               | If you retained the default location for storing flows (`<installation-directory>/conf/`), copy _flow.json.gz_ from the existing to the new NiFi base install `conf` directory. If you stored flows to an external location via _nifi.properties_, update the property `nifi.flow.configuration.file` to point there.
 
-If you are encrypting sensitive component properties in your dataflow via the sensitive properties key in _nifi.properties_, make sure the same key is used when copying over your _flow.xml.gz_.  If you need to change the key, see the <<sensitive_flow_migration>> section below.
+If you are encrypting sensitive component properties in your dataflow via the sensitive properties key in _nifi.properties_, make sure the same key is used when copying over your _flow.json.gz_.  If you need to change the key, see the <<sensitive_flow_migration>> section below.
 
 .8+|_nifi.properties_
 |Use the existing _nifi.properties_ to populate the same properties in the new NiFi file.
@@ -4225,15 +4233,17 @@ If you are encrypting sensitive component properties in your dataflow via the se
 
 If the below properties point to directories inside the NiFi base installation path, you must copy the target directories to the new NiFi. Stop your existing NiFi installation before you do this.
 
-|`nifi.flow.configuration.file=`
+|`nifi.flow.configuration.json.file=`
 
-If you have retained the default value (`./conf/flow.xml.gz`), copy _flow.xml.gz_ from the existing to the new NiFi base install conf directory.
+If you have retained the default value (`./conf/flow.json.gz`), copy _flow.json.gz_ from the existing to the new NiFi base install conf directory.
 
 If you stored flows to an external location, update the property value to point there.
 
+
+
 |`nifi.flow.configuration.archive.dir=`
 
-Same applies as above if you want to retain archived copies of the _flow.xml.gz_.
+Same applies as above if you want to retain archived copies of the _flow.json.gz_.
 
 |`nifi.database.directory=`
 
@@ -4285,22 +4295,23 @@ WARNING: Double check all configured properties for typos.
 [[sensitive_flow_migration]]
 ==== Migrating a Flow with Sensitive Properties
 
-When a value is set for `nifi.sensitive.props.key` in _nifi.properties_, the specified key is used to encrypt sensitive properties in the flow (e.g. password fields in components). If the key needs to change, the Encrypt-Config tool in the NiFi Toolkit can migrate the sensitive properties key and update the _flow.xml.gz_. Specifically, Encrypt-Config:
+When a value is set for `nifi.sensitive.props.key` in _nifi.properties_, the specified key is used to encrypt sensitive properties in the flow (e.g. password fields in components). If the key needs to change, the Encrypt-Config tool in the NiFi Toolkit can migrate the sensitive properties key and update the _flow.json.gz_. Specifically, Encrypt-Config:
 
-1. Reads the existing _flow.xml.gz_ and decrypts the sensitive values using the current key.
+1. Reads the existing _flow.json.gz_ and decrypts the sensitive values using the current key.
 2. Encrypts all the sensitive values with a specified new key.
-3. Updates the _nifi.properties_ and _flow.xml.gz_ files or creates new versions of them.
+3. Updates the _nifi.properties_ and _flow.json.gz_ files or creates new versions of them.
 
 As an example, assume version 1.9.2 is the existing NiFi instance and the sensitive properties key is set to `password`. The goal is to move the 1.9.2 _flow.xml.gz_ to a 1.10.0 instance with a new sensitive properties key: `new_password`. Running the following Encrypt-Config command would read in the _flow.xml.gz_ and _nifi.properties_ files from 1.9.2 using the original sensitive properties key and write out new versions in 1.10.0 with the sensitive properties encrypted with the new password:
 
 ```
-$ ./nifi-toolkit-1.10.0/bin/encrypt-config.sh -f /path/to/nifi/nifi-1.9.2/conf/flow.xml.gz -g /path/to/nifi/nifi-1.10.0/conf/flow.xml.gz -s new_password -n /path/to/nifi/nifi-1.9.2/conf/nifi.properties -o /path/to/nifi/nifi-1.10.0/conf/nifi.properties -x
+$ ./nifi-toolkit-1.10.0/bin/encrypt-config.sh -f /path/to/nifi/nifi-1.9.2/conf/flow.json.gz -g /path/to/nifi/nifi-1.10.0/conf/flow.json.gz -s new_password -n /path/to/nifi/nifi-1.9.2/conf/nifi
+.properties -o /path/to/nifi/nifi-1.10.0/conf/nifi.properties -x
 ```
 
 where:
 
-* `-f` specifies the source _flow.xml.gz_ (nifi-1.9.2)
-* `-g` specifies the destination _flow.xml.gz_ (nifi-1.10.0)
+* `-f` specifies the source _flow.json.gz_ (nifi-1.9.2)
+* `-g` specifies the destination _flow.json.gz_ (nifi-1.10.0)
 * `-s` specifies the new sensitive properties key (`new_password`)
 * `-n` specifies the source _nifi.properties_ (nifi-1.9.2)
 * `-o` specifies the destination _nifi.properties_ (nifi-1.10.0)
@@ -4312,7 +4323,7 @@ For more information see the <<toolkit-guide.adoc#encrypt_config_tool,Encrypt-Co
 
 Starting with version 1.14.0, NiFi requires a value for 'nifi.sensitive.props.key' in _nifi.properties_.
 
-The following command can be used to read an existing _flow.xml.gz_ configuration and set a new sensitive properties key in _nifi.properties_:
+The following command can be used to read an existing _flow.json.gz_ configuration and set a new sensitive properties key in _nifi.properties_:
 
 ```
 $ ./bin/nifi.sh set-sensitive-properties-key <sensitivePropertiesKey>
diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/authorization/AbstractPolicyBasedAuthorizer.java b/nifi-framework-api/src/main/java/org/apache/nifi/authorization/AbstractPolicyBasedAuthorizer.java
index 0b86543..6a4fa17 100644
--- a/nifi-framework-api/src/main/java/org/apache/nifi/authorization/AbstractPolicyBasedAuthorizer.java
+++ b/nifi-framework-api/src/main/java/org/apache/nifi/authorization/AbstractPolicyBasedAuthorizer.java
@@ -20,7 +20,6 @@ import org.apache.nifi.authorization.exception.AuthorizationAccessException;
 import org.apache.nifi.authorization.exception.AuthorizerCreationException;
 import org.apache.nifi.authorization.exception.AuthorizerDestructionException;
 import org.apache.nifi.authorization.exception.UninheritableAuthorizationsException;
-import org.apache.nifi.security.xml.XmlUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.w3c.dom.Document;
@@ -30,6 +29,7 @@ import org.w3c.dom.NodeList;
 import org.xml.sax.SAXException;
 
 import javax.xml.parsers.DocumentBuilder;
+import javax.xml.parsers.DocumentBuilderFactory;
 import javax.xml.parsers.ParserConfigurationException;
 import javax.xml.stream.XMLOutputFactory;
 import javax.xml.stream.XMLStreamException;
@@ -49,6 +49,10 @@ import java.util.Set;
  */
 public abstract class AbstractPolicyBasedAuthorizer implements ManagedAuthorizer {
     private static final Logger logger = LoggerFactory.getLogger(AbstractPolicyBasedAuthorizer.class);
+    private static final String DISALLOW_DOCTYPES = "http://apache.org/xml/features/disallow-doctype-decl";
+    private static final String ALLOW_EXTERNAL_GENERAL_ENTITIES = "http://xml.org/sax/features/external-general-entities";
+    private static final String ALLOW_EXTERNAL_PARAM_ENTITIES = "http://xml.org/sax/features/external-parameter-entities";
+    private static final String ALLOW_EXTERNAL_DTD = "http://apache.org/xml/features/nonvalidating/load-external-dtd";
 
     static final XMLOutputFactory XML_OUTPUT_FACTORY = XMLOutputFactory.newInstance();
 
@@ -423,7 +427,7 @@ public abstract class AbstractPolicyBasedAuthorizer implements ManagedAuthorizer
 
         final byte[] fingerprintBytes = fingerprint.getBytes(StandardCharsets.UTF_8);
         try (final ByteArrayInputStream in = new ByteArrayInputStream(fingerprintBytes)) {
-            final DocumentBuilder docBuilder = XmlUtils.createSafeDocumentBuilder(null);
+            final DocumentBuilder docBuilder = createSafeDocumentBuilder();
             final Document document = docBuilder.parse(in);
             final Element rootElement = document.getDocumentElement();
 
@@ -454,6 +458,22 @@ public abstract class AbstractPolicyBasedAuthorizer implements ManagedAuthorizer
         return new PoliciesUsersAndGroups(accessPolicies, users, groups);
     }
 
+    public static DocumentBuilder createSafeDocumentBuilder() throws ParserConfigurationException {
+        final DocumentBuilderFactory docFactory = DocumentBuilderFactory.newInstance();
+        docFactory.setSchema(null);
+        docFactory.setNamespaceAware(true);
+
+        // Disable DTDs and external entities to protect against XXE
+        docFactory.setAttribute(DISALLOW_DOCTYPES, true);
+        docFactory.setAttribute(ALLOW_EXTERNAL_DTD, false);
+        docFactory.setAttribute(ALLOW_EXTERNAL_GENERAL_ENTITIES, false);
+        docFactory.setAttribute(ALLOW_EXTERNAL_PARAM_ENTITIES, false);
+        docFactory.setXIncludeAware(false);
+        docFactory.setExpandEntityReferences(false);
+
+        return docFactory.newDocumentBuilder();
+    }
+
     private User parseUser(final Element element) {
         final User.Builder builder = new User.Builder()
                 .identifier(element.getAttribute(IDENTIFIER_ATTR))
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/main/java/org/apache/nifi/authorization/FileAccessPolicyProvider.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/main/java/org/apache/nifi/authorization/FileAccessPolicyProvider.java
index bb9839e..dcd23d7 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/main/java/org/apache/nifi/authorization/FileAccessPolicyProvider.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/main/java/org/apache/nifi/authorization/FileAccessPolicyProvider.java
@@ -16,40 +16,6 @@
  */
 package org.apache.nifi.authorization;
 
-import java.io.ByteArrayInputStream;
-import java.io.File;
-import java.io.IOException;
-import java.io.StringWriter;
-import java.nio.charset.StandardCharsets;
-import java.text.DateFormat;
-import java.text.SimpleDateFormat;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.Date;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-import javax.xml.XMLConstants;
-import javax.xml.bind.JAXBContext;
-import javax.xml.bind.JAXBElement;
-import javax.xml.bind.JAXBException;
-import javax.xml.bind.Marshaller;
-import javax.xml.bind.Unmarshaller;
-import javax.xml.parsers.DocumentBuilder;
-import javax.xml.parsers.ParserConfigurationException;
-import javax.xml.stream.XMLOutputFactory;
-import javax.xml.stream.XMLStreamException;
-import javax.xml.stream.XMLStreamReader;
-import javax.xml.stream.XMLStreamWriter;
-import javax.xml.transform.stream.StreamSource;
-import javax.xml.validation.Schema;
-import javax.xml.validation.SchemaFactory;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.authorization.annotation.AuthorizerContext;
 import org.apache.nifi.authorization.exception.AuthorizationAccessException;
@@ -79,6 +45,41 @@ import org.w3c.dom.Node;
 import org.w3c.dom.NodeList;
 import org.xml.sax.SAXException;
 
+import javax.xml.XMLConstants;
+import javax.xml.bind.JAXBContext;
+import javax.xml.bind.JAXBElement;
+import javax.xml.bind.JAXBException;
+import javax.xml.bind.Marshaller;
+import javax.xml.bind.Unmarshaller;
+import javax.xml.parsers.DocumentBuilder;
+import javax.xml.parsers.ParserConfigurationException;
+import javax.xml.stream.XMLOutputFactory;
+import javax.xml.stream.XMLStreamException;
+import javax.xml.stream.XMLStreamReader;
+import javax.xml.stream.XMLStreamWriter;
+import javax.xml.transform.stream.StreamSource;
+import javax.xml.validation.Schema;
+import javax.xml.validation.SchemaFactory;
+import java.io.ByteArrayInputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.StringWriter;
+import java.nio.charset.StandardCharsets;
+import java.text.DateFormat;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Date;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
 public class FileAccessPolicyProvider implements ConfigurableAccessPolicyProvider {
 
     private static final Logger logger = LoggerFactory.getLogger(FileAccessPolicyProvider.class);
@@ -649,7 +650,16 @@ public class FileAccessPolicyProvider implements ConfigurableAccessPolicyProvide
      */
     private void parseFlow() throws SAXException {
         final FlowParser flowParser = new FlowParser();
-        final FlowInfo flowInfo = flowParser.parse(properties.getFlowConfigurationFile());
+
+        final File flowConfigurationFile;
+        final File jsonFile = properties.getFlowConfigurationJsonFile();
+        if (jsonFile.exists()) {
+            flowConfigurationFile = jsonFile;
+        } else {
+            flowConfigurationFile = properties.getFlowConfigurationFile();
+        }
+
+        final FlowInfo flowInfo = flowParser.parse(flowConfigurationFile);
 
         if (flowInfo != null) {
             rootGroupId = flowInfo.getRootGroupId();
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/test/java/org/apache/nifi/authorization/FileAccessPolicyProviderTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/test/java/org/apache/nifi/authorization/FileAccessPolicyProviderTest.java
index 2f50051..92ac0b4 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/test/java/org/apache/nifi/authorization/FileAccessPolicyProviderTest.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/test/java/org/apache/nifi/authorization/FileAccessPolicyProviderTest.java
@@ -145,6 +145,7 @@ public class FileAccessPolicyProviderTest {
     private File restoreAuthorizations;
     private File restoreTenants;
     private File flow;
+    private File flowJson;
     private File flowNoPorts;
     private File flowWithDns;
 
@@ -171,6 +172,8 @@ public class FileAccessPolicyProviderTest {
         flow = new File("src/test/resources/flow.xml.gz");
         FileUtils.ensureDirectoryExistAndCanAccess(flow.getParentFile());
 
+        flowJson = new File("src/test/resources/flow.json.gz");
+
         flowNoPorts = new File("src/test/resources/flow-no-ports.xml.gz");
         FileUtils.ensureDirectoryExistAndCanAccess(flowNoPorts.getParentFile());
 
@@ -180,6 +183,7 @@ public class FileAccessPolicyProviderTest {
         properties = mock(NiFiProperties.class);
         when(properties.getRestoreDirectory()).thenReturn(restoreAuthorizations.getParentFile());
         when(properties.getFlowConfigurationFile()).thenReturn(flow);
+        when(properties.getFlowConfigurationJsonFile()).thenReturn(flowJson);
 
         userGroupProvider = new FileUserGroupProvider();
         userGroupProvider.setNiFiProperties(properties);
@@ -465,6 +469,7 @@ public class FileAccessPolicyProviderTest {
         properties = getNiFiProperties(props);
         when(properties.getRestoreDirectory()).thenReturn(restoreAuthorizations.getParentFile());
         when(properties.getFlowConfigurationFile()).thenReturn(flowWithDns);
+        when(properties.getFlowConfigurationJsonFile()).thenReturn(flowJson);
 
         userGroupProvider.setNiFiProperties(properties);
         accessPolicyProvider.setNiFiProperties(properties);
@@ -577,6 +582,7 @@ public class FileAccessPolicyProviderTest {
         properties = mock(NiFiProperties.class);
         when(properties.getRestoreDirectory()).thenReturn(restoreAuthorizations.getParentFile());
         when(properties.getFlowConfigurationFile()).thenReturn(new File("src/test/resources/does-not-exist.xml.gz"));
+        when(properties.getFlowConfigurationJsonFile()).thenReturn(flowJson);
 
         userGroupProvider.setNiFiProperties(properties);
         accessPolicyProvider.setNiFiProperties(properties);
@@ -617,6 +623,7 @@ public class FileAccessPolicyProviderTest {
         properties = mock(NiFiProperties.class);
         when(properties.getRestoreDirectory()).thenReturn(restoreAuthorizations.getParentFile());
         when(properties.getFlowConfigurationFile()).thenReturn(null);
+        when(properties.getFlowConfigurationJsonFile()).thenReturn(flowJson);
 
         userGroupProvider.setNiFiProperties(properties);
         accessPolicyProvider.setNiFiProperties(properties);
@@ -660,6 +667,7 @@ public class FileAccessPolicyProviderTest {
         properties = getNiFiProperties(props);
         when(properties.getRestoreDirectory()).thenReturn(restoreAuthorizations.getParentFile());
         when(properties.getFlowConfigurationFile()).thenReturn(flow);
+        when(properties.getFlowConfigurationJsonFile()).thenReturn(flowJson);
 
         userGroupProvider.setNiFiProperties(properties);
         accessPolicyProvider.setNiFiProperties(properties);
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/test/java/org/apache/nifi/authorization/FileAuthorizerTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/test/java/org/apache/nifi/authorization/FileAuthorizerTest.java
index 1b8918e..b238d24 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/test/java/org/apache/nifi/authorization/FileAuthorizerTest.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/test/java/org/apache/nifi/authorization/FileAuthorizerTest.java
@@ -156,6 +156,7 @@ public class FileAuthorizerTest {
     private File restoreAuthorizations;
     private File restoreTenants;
     private File flow;
+    private File flowJson;
     private File flowNoPorts;
     private File flowWithDns;
 
@@ -187,6 +188,8 @@ public class FileAuthorizerTest {
         flow = new File("src/test/resources/flow.xml.gz");
         FileUtils.ensureDirectoryExistAndCanAccess(flow.getParentFile());
 
+        flowJson = new File("src/test/resources/flow.json.gz");
+
         flowNoPorts = new File("src/test/resources/flow-no-ports.xml.gz");
         FileUtils.ensureDirectoryExistAndCanAccess(flowNoPorts.getParentFile());
 
@@ -196,6 +199,7 @@ public class FileAuthorizerTest {
         properties = mock(NiFiProperties.class);
         when(properties.getRestoreDirectory()).thenReturn(restoreAuthorizations.getParentFile());
         when(properties.getFlowConfigurationFile()).thenReturn(flow);
+        when(properties.getFlowConfigurationJsonFile()).thenReturn(flowJson);
 
         configurationContext = mock(AuthorizerConfigurationContext.class);
         when(configurationContext.getProperty(Mockito.eq(FileAccessPolicyProvider.PROP_AUTHORIZATIONS_FILE))).thenReturn(new StandardPropertyValue(primaryAuthorizations.getPath(), null,
@@ -460,6 +464,7 @@ public class FileAuthorizerTest {
         properties = getNiFiProperties(props);
         when(properties.getRestoreDirectory()).thenReturn(restoreAuthorizations.getParentFile());
         when(properties.getFlowConfigurationFile()).thenReturn(flowWithDns);
+        when(properties.getFlowConfigurationJsonFile()).thenReturn(flowJson);
         authorizer.setNiFiProperties(properties);
 
         when(configurationContext.getProperty(Mockito.eq(FileAuthorizer.PROP_LEGACY_AUTHORIZED_USERS_FILE)))
@@ -586,6 +591,7 @@ public class FileAuthorizerTest {
         properties = mock(NiFiProperties.class);
         when(properties.getRestoreDirectory()).thenReturn(restoreAuthorizations.getParentFile());
         when(properties.getFlowConfigurationFile()).thenReturn(new File("src/test/resources/does-not-exist.xml.gz"));
+        when(properties.getFlowConfigurationJsonFile()).thenReturn(flowJson);
         authorizer.setNiFiProperties(properties);
 
         final String adminIdentity = "admin-user";
@@ -624,6 +630,7 @@ public class FileAuthorizerTest {
         properties = mock(NiFiProperties.class);
         when(properties.getRestoreDirectory()).thenReturn(restoreAuthorizations.getParentFile());
         when(properties.getFlowConfigurationFile()).thenReturn(null);
+        when(properties.getFlowConfigurationJsonFile()).thenReturn(flowJson);
         authorizer.setNiFiProperties(properties);
 
         final String adminIdentity = "admin-user";
@@ -665,6 +672,7 @@ public class FileAuthorizerTest {
         properties = getNiFiProperties(props);
         when(properties.getRestoreDirectory()).thenReturn(restoreAuthorizations.getParentFile());
         when(properties.getFlowConfigurationFile()).thenReturn(flow);
+        when(properties.getFlowConfigurationJsonFile()).thenReturn(flowJson);
         authorizer.setNiFiProperties(properties);
 
         final String adminIdentity = "CN=localhost, OU=Apache NiFi, O=Apache, L=Santa Monica, ST=CA, C=US";
@@ -759,6 +767,7 @@ public class FileAuthorizerTest {
         properties = getNiFiProperties(props);
         when(properties.getRestoreDirectory()).thenReturn(restoreAuthorizations.getParentFile());
         when(properties.getFlowConfigurationFile()).thenReturn(flow);
+        when(properties.getFlowConfigurationJsonFile()).thenReturn(flowJson);
         authorizer.setNiFiProperties(properties);
 
         final String adminIdentity = "CN=user1, OU=Apache NiFi, O=Apache, L=Santa Monica, ST=CA, C=US";
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/pom.xml
index f783676..62f5517 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/pom.xml
@@ -47,7 +47,7 @@
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-framework-core-api</artifactId>
         </dependency>
-        
+
         <!-- spring dependencies -->
         <dependency>
             <groupId>org.springframework</groupId>
@@ -61,6 +61,11 @@
             <groupId>org.springframework</groupId>
             <artifactId>spring-context</artifactId>
         </dependency>
+        <dependency>
+            <groupId>com.fasterxml.jackson.module</groupId>
+            <artifactId>jackson-module-jaxb-annotations</artifactId>
+            <version>${jackson.version}</version>
+        </dependency>
     </dependencies>
     <profiles>
         <profile>
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/StandardDataFlow.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/StandardDataFlow.java
index 3ac2052..52d8dff 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/StandardDataFlow.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/StandardDataFlow.java
@@ -16,18 +16,11 @@
  */
 package org.apache.nifi.cluster.protocol;
 
-import java.io.ByteArrayInputStream;
-import java.io.IOException;
-import java.io.Serializable;
-import java.net.URL;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.Set;
-import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
-import javax.xml.parsers.DocumentBuilder;
-import javax.xml.parsers.ParserConfigurationException;
+import com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.module.jaxb.JaxbAnnotationIntrospector;
 import org.apache.nifi.cluster.protocol.jaxb.message.DataFlowAdapter;
+import org.apache.nifi.controller.flow.VersionedDataflow;
 import org.apache.nifi.controller.serialization.FlowSerializationException;
 import org.apache.nifi.security.xml.XmlUtils;
 import org.slf4j.Logger;
@@ -37,6 +30,18 @@ import org.xml.sax.SAXException;
 import org.xml.sax.SAXParseException;
 import org.xml.sax.helpers.DefaultHandler;
 
+import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
+import javax.xml.parsers.DocumentBuilder;
+import javax.xml.parsers.ParserConfigurationException;
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.Serializable;
+import java.net.URL;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+
 /**
  * Represents a dataflow, which includes the raw bytes of the flow.xml and
  * whether processors should be started automatically at application startup.
@@ -53,6 +58,8 @@ public class StandardDataFlow implements Serializable, DataFlow {
     private final byte[] authorizerFingerprint;
     private final Set<String> missingComponentIds;
     private Document flowDocument;
+    private VersionedDataflow versionedDataflow;
+
 
     /**
      * Constructs an instance.
@@ -100,6 +107,15 @@ public class StandardDataFlow implements Serializable, DataFlow {
     }
 
     @Override
+    public synchronized VersionedDataflow getVersionedDataflow() {
+        if (versionedDataflow == null) {
+            versionedDataflow = parseVersionedDataflow(flow);
+        }
+
+        return versionedDataflow;
+    }
+
+    @Override
     public byte[] getSnippets() {
         return snippetBytes;
     }
@@ -136,4 +152,29 @@ public class StandardDataFlow implements Serializable, DataFlow {
             throw new FlowSerializationException(ex);
         }
     }
+
+    private VersionedDataflow parseVersionedDataflow(final byte[] flow) {
+        if (flow == null || flow.length == 0) {
+            return null;
+        }
+
+        try {
+            final ObjectMapper objectMapper = new ObjectMapper();
+            objectMapper.setAnnotationIntrospector(new JaxbAnnotationIntrospector(objectMapper.getTypeFactory()));
+            objectMapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
+
+            final VersionedDataflow versionedDataflow = objectMapper.readValue(flow, VersionedDataflow.class);
+            return versionedDataflow;
+        } catch (final Exception e) {
+            throw new FlowSerializationException("Could not parse flow as a VersionedDataflow", e);
+        }
+    }
+
+    public boolean isXml() {
+        if (flow == null || flow.length == 0) {
+            return true;
+        }
+
+        return flow[0] == '<';
+    }
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/flow/PopularVoteFlowElection.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/flow/PopularVoteFlowElection.java
index b9df55e..5e88021 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/flow/PopularVoteFlowElection.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/flow/PopularVoteFlowElection.java
@@ -17,7 +17,12 @@
 
 package org.apache.nifi.cluster.coordination.flow;
 
-import static java.util.Objects.requireNonNull;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.nifi.cluster.protocol.DataFlow;
+import org.apache.nifi.cluster.protocol.NodeIdentifier;
+import org.apache.nifi.controller.serialization.StandardFlowSynchronizer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.nio.charset.StandardCharsets;
 import java.util.Collections;
@@ -30,13 +35,6 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.stream.Collectors;
 
-import org.apache.nifi.cluster.protocol.DataFlow;
-import org.apache.nifi.cluster.protocol.NodeIdentifier;
-import org.apache.nifi.controller.StandardFlowSynchronizer;
-import org.apache.nifi.fingerprint.FingerprintFactory;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 /**
  * <p>
  * An implementation of {@link FlowElection} that waits until either a maximum amount of time has elapsed
@@ -53,14 +51,13 @@ public class PopularVoteFlowElection implements FlowElection {
 
     private final long maxWaitNanos;
     private final Integer maxNodes;
-    private final FingerprintFactory fingerprintFactory;
 
     private volatile Long startNanos = null;
     private volatile DataFlow electedDataFlow = null;
 
     private final Map<String, FlowCandidate> candidateByFingerprint = new HashMap<>();
 
-    public PopularVoteFlowElection(final long maxWait, final TimeUnit maxWaitPeriod, final Integer maxNodes, final FingerprintFactory fingerprintFactory) {
+    public PopularVoteFlowElection(final long maxWait, final TimeUnit maxWaitPeriod, final Integer maxNodes) {
         this.maxWaitNanos = maxWaitPeriod.toNanos(maxWait);
         if (maxWaitNanos < 1) {
             throw new IllegalArgumentException("Maximum wait time to elect Cluster Flow cannot be less than 1 nanosecond");
@@ -70,8 +67,6 @@ public class PopularVoteFlowElection implements FlowElection {
         if (maxNodes != null && maxNodes < 1) {
             throw new IllegalArgumentException("Maximum number of nodes to wait on before electing Cluster Flow cannot be less than 1");
         }
-
-        this.fingerprintFactory = requireNonNull(fingerprintFactory);
     }
 
     @Override
@@ -141,7 +136,7 @@ public class PopularVoteFlowElection implements FlowElection {
     }
 
     private String fingerprint(final DataFlow dataFlow) {
-        final String flowFingerprint = fingerprintFactory.createFingerprint(dataFlow.getFlow());
+        final String flowFingerprint = DigestUtils.sha256Hex(dataFlow.getFlow());
         final String authFingerprint = dataFlow.getAuthorizerFingerprint() == null ? "" : new String(dataFlow.getAuthorizerFingerprint(), StandardCharsets.UTF_8);
         final String candidateFingerprint = flowFingerprint + authFingerprint;
 
@@ -244,7 +239,7 @@ public class PopularVoteFlowElection implements FlowElection {
         }
 
         public boolean isFlowEmpty() {
-            return StandardFlowSynchronizer.isEmpty(dataFlow);
+            return StandardFlowSynchronizer.isFlowEmpty(dataFlow);
         }
 
         public Set<NodeIdentifier> getNodes() {
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/flow/PopularVoteFlowElectionFactoryBean.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/flow/PopularVoteFlowElectionFactoryBean.java
index b892e82..b3225b0 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/flow/PopularVoteFlowElectionFactoryBean.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/flow/PopularVoteFlowElectionFactoryBean.java
@@ -17,12 +17,6 @@
 
 package org.apache.nifi.cluster.coordination.flow;
 
-import org.apache.nifi.encrypt.PropertyEncryptor;
-import org.apache.nifi.encrypt.PropertyEncryptorFactory;
-import org.apache.nifi.encrypt.SensitiveValueEncoder;
-import org.apache.nifi.encrypt.StandardSensitiveValueEncoder;
-import org.apache.nifi.fingerprint.FingerprintFactory;
-import org.apache.nifi.nar.ExtensionManager;
 import org.apache.nifi.util.FormatUtils;
 import org.apache.nifi.util.NiFiProperties;
 import org.slf4j.Logger;
@@ -34,7 +28,6 @@ import java.util.concurrent.TimeUnit;
 public class PopularVoteFlowElectionFactoryBean implements FactoryBean<PopularVoteFlowElection> {
     private static final Logger logger = LoggerFactory.getLogger(PopularVoteFlowElectionFactoryBean.class);
     private NiFiProperties properties;
-    private ExtensionManager extensionManager;
 
     @Override
     public PopularVoteFlowElection getObject() {
@@ -49,10 +42,7 @@ public class PopularVoteFlowElectionFactoryBean implements FactoryBean<PopularVo
         }
 
         final Integer maxNodes = properties.getFlowElectionMaxCandidates();
-        final PropertyEncryptor encryptor = PropertyEncryptorFactory.getPropertyEncryptor(properties);
-        final SensitiveValueEncoder sensitiveValueEncoder = new StandardSensitiveValueEncoder(properties);
-        final FingerprintFactory fingerprintFactory = new FingerprintFactory(encryptor, extensionManager, sensitiveValueEncoder);
-        return new PopularVoteFlowElection(maxWaitMillis, TimeUnit.MILLISECONDS, maxNodes, fingerprintFactory);
+        return new PopularVoteFlowElection(maxWaitMillis, TimeUnit.MILLISECONDS, maxNodes);
     }
 
     @Override
@@ -68,8 +58,4 @@ public class PopularVoteFlowElectionFactoryBean implements FactoryBean<PopularVo
     public void setProperties(final NiFiProperties properties) {
         this.properties = properties;
     }
-
-    public void setExtensionManager(ExtensionManager extensionManager) {
-        this.extensionManager = extensionManager;
-    }
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/replication/RequestReplicator.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/replication/RequestReplicator.java
index 6af6111..1965ad9 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/replication/RequestReplicator.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/replication/RequestReplicator.java
@@ -17,15 +17,14 @@
 
 package org.apache.nifi.cluster.coordination.http.replication;
 
-import java.net.URI;
-import java.util.Map;
-import java.util.Set;
-
 import org.apache.nifi.authorization.user.NiFiUser;
 import org.apache.nifi.cluster.manager.exception.ConnectingNodeMutableRequestException;
-import org.apache.nifi.cluster.manager.exception.DisconnectedNodeMutableRequestException;
 import org.apache.nifi.cluster.protocol.NodeIdentifier;
 
+import java.net.URI;
+import java.util.Map;
+import java.util.Set;
+
 public interface RequestReplicator {
 
     public static final String REQUEST_TRANSACTION_ID_HEADER = "X-RequestTransactionId";
@@ -86,7 +85,6 @@ public interface RequestReplicator {
      * @param headers any HTTP headers
      * @return an AsyncClusterResponse that indicates the current status of the request and provides an identifier for obtaining an updated response later
      * @throws ConnectingNodeMutableRequestException   if the request attempts to modify the flow and there is a node that is in the CONNECTING state
-     * @throws DisconnectedNodeMutableRequestException if the request attempts to modify the flow and there is a node that is in the DISCONNECTED state
      */
     AsyncClusterResponse replicate(String method, URI uri, Object entity, Map<String, String> headers);
 
@@ -103,7 +101,6 @@ public interface RequestReplicator {
      * @param headers any HTTP headers
      * @return an AsyncClusterResponse that indicates the current status of the request and provides an identifier for obtaining an updated response later
      * @throws ConnectingNodeMutableRequestException if the request attempts to modify the flow and there is a node that is in the CONNECTING state
-     * @throws DisconnectedNodeMutableRequestException if the request attempts to modify the flow and there is a node that is in the DISCONNECTED state
      */
     AsyncClusterResponse replicate(NiFiUser user, String method, URI uri, Object entity, Map<String, String> headers);
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/replication/ThreadPoolRequestReplicator.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/replication/ThreadPoolRequestReplicator.java
index 555ae9e..343bc56 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/replication/ThreadPoolRequestReplicator.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/replication/ThreadPoolRequestReplicator.java
@@ -28,10 +28,8 @@ import org.apache.nifi.cluster.coordination.node.NodeConnectionState;
 import org.apache.nifi.cluster.coordination.node.NodeConnectionStatus;
 import org.apache.nifi.cluster.manager.NodeResponse;
 import org.apache.nifi.cluster.manager.exception.ConnectingNodeMutableRequestException;
-import org.apache.nifi.cluster.manager.exception.DisconnectedNodeMutableRequestException;
 import org.apache.nifi.cluster.manager.exception.IllegalClusterStateException;
 import org.apache.nifi.cluster.manager.exception.NoConnectedNodesException;
-import org.apache.nifi.cluster.manager.exception.OffloadedNodeMutableRequestException;
 import org.apache.nifi.cluster.manager.exception.UnknownNodeException;
 import org.apache.nifi.cluster.manager.exception.UriConstructionException;
 import org.apache.nifi.cluster.protocol.NodeIdentifier;
@@ -170,42 +168,6 @@ public class ThreadPoolRequestReplicator implements RequestReplicator {
 
         // If the request is mutable, ensure that all nodes are connected.
         if (mutable) {
-            final List<NodeIdentifier> offloaded = stateMap.get(NodeConnectionState.OFFLOADED);
-            if (offloaded != null && !offloaded.isEmpty()) {
-                if (offloaded.size() == 1) {
-                    throw new OffloadedNodeMutableRequestException("Node " + offloaded.iterator().next() + " is currently offloaded");
-                } else {
-                    throw new OffloadedNodeMutableRequestException(offloaded.size() + " Nodes are currently offloaded");
-                }
-            }
-
-            final List<NodeIdentifier> offloading = stateMap.get(NodeConnectionState.OFFLOADING);
-            if (offloading != null && !offloading.isEmpty()) {
-                if (offloading.size() == 1) {
-                    throw new OffloadedNodeMutableRequestException("Node " + offloading.iterator().next() + " is currently offloading");
-                } else {
-                    throw new OffloadedNodeMutableRequestException(offloading.size() + " Nodes are currently offloading");
-                }
-            }
-
-            final List<NodeIdentifier> disconnected = stateMap.get(NodeConnectionState.DISCONNECTED);
-            if (disconnected != null && !disconnected.isEmpty()) {
-                if (disconnected.size() == 1) {
-                    throw new DisconnectedNodeMutableRequestException("Node " + disconnected.iterator().next() + " is currently disconnected");
-                } else {
-                    throw new DisconnectedNodeMutableRequestException(disconnected.size() + " Nodes are currently disconnected");
-                }
-            }
-
-            final List<NodeIdentifier> disconnecting = stateMap.get(NodeConnectionState.DISCONNECTING);
-            if (disconnecting != null && !disconnecting.isEmpty()) {
-                if (disconnecting.size() == 1) {
-                    throw new DisconnectedNodeMutableRequestException("Node " + disconnecting.iterator().next() + " is currently disconnecting");
-                } else {
-                    throw new DisconnectedNodeMutableRequestException(disconnecting.size() + " Nodes are currently disconnecting");
-                }
-            }
-
             final List<NodeIdentifier> connecting = stateMap.get(NodeConnectionState.CONNECTING);
             if (connecting != null && !connecting.isEmpty()) {
                 if (connecting.size() == 1) {
@@ -679,9 +641,6 @@ public class ThreadPoolRequestReplicator implements RequestReplicator {
         // check that the request can be applied
         if (mutableRequest) {
             final Map<NodeConnectionState, List<NodeIdentifier>> connectionStates = clusterCoordinator.getConnectionStates();
-            if (connectionStates.containsKey(NodeConnectionState.DISCONNECTED) || connectionStates.containsKey(NodeConnectionState.DISCONNECTING)) {
-                throw new DisconnectedNodeMutableRequestException("Received a mutable request [" + httpMethod + " " + uriPath + "] while a node is disconnected from the cluster");
-            }
 
             if (connectionStates.containsKey(NodeConnectionState.CONNECTING)) {
                 // if any node is connecting and a request can change the flow, then we throw an exception
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/resources/nifi-cluster-manager-context.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/resources/nifi-cluster-manager-context.xml
index a4c2573..2d91faa 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/resources/nifi-cluster-manager-context.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/resources/nifi-cluster-manager-context.xml
@@ -43,7 +43,6 @@
 
     <bean id="flowElection" class="org.apache.nifi.cluster.coordination.flow.PopularVoteFlowElectionFactoryBean">
         <property name="properties" ref="nifiProperties" />
-        <property name="extensionManager" ref="extensionManager" />
     </bean>
 
     <!-- Cluster Coordinator -->
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/groovy/org/apache/nifi/cluster/coordination/flow/PopularVoteFlowElectionFactoryBeanTest.groovy b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/groovy/org/apache/nifi/cluster/coordination/flow/PopularVoteFlowElectionFactoryBeanTest.groovy
deleted file mode 100644
index eb29539..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/groovy/org/apache/nifi/cluster/coordination/flow/PopularVoteFlowElectionFactoryBeanTest.groovy
+++ /dev/null
@@ -1,87 +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.cluster.coordination.flow
-
-import org.apache.nifi.encrypt.PropertyEncryptor
-import org.apache.nifi.encrypt.PropertyEncryptorFactory
-
-import org.apache.nifi.security.util.EncryptionMethod
-import org.apache.nifi.util.NiFiProperties
-import org.junit.Before
-import org.junit.BeforeClass
-import org.junit.Test
-import org.junit.runner.RunWith
-import org.junit.runners.JUnit4
-import org.slf4j.Logger
-import org.slf4j.LoggerFactory
-
-@RunWith(JUnit4.class)
-class PopularVoteFlowElectionFactoryBeanTest extends GroovyTestCase {
-    private static final Logger logger = LoggerFactory.getLogger(PopularVoteFlowElectionFactoryBeanTest.class)
-
-    private static final EncryptionMethod DEFAULT_ENCRYPTION_METHOD = EncryptionMethod.MD5_256AES
-
-    private final String DEFAULT_SENSITIVE_PROPS_KEY = String.class.getName()
-
-    @BeforeClass
-    static void setUpOnce() {
-        logger.metaClass.methodMissing = { String name, args ->
-            logger.info("[${name?.toUpperCase()}] ${(args as List).join(" ")}")
-        }
-    }
-
-    @Before
-    void setUp() {
-        super.setUp()
-
-    }
-
-    NiFiProperties mockProperties(Map<String, String> defaults = [:]) {
-        def mockProps = new NiFiProperties(new Properties([
-                (NiFiProperties.SENSITIVE_PROPS_ALGORITHM):DEFAULT_ENCRYPTION_METHOD.algorithm,
-        ] + defaults))
-
-        mockProps
-    }
-
-    @Test
-    void testGetObjectShouldPopulateSensitivePropsKeyIfPresent() {
-        // Arrange
-        final String REVERSE_KEY = DEFAULT_SENSITIVE_PROPS_KEY.reverse()
-
-        PopularVoteFlowElectionFactoryBean electionFactoryBean = new PopularVoteFlowElectionFactoryBean()
-        electionFactoryBean.properties = mockProperties([(NiFiProperties.SENSITIVE_PROPS_KEY): REVERSE_KEY])
-
-        final PropertyEncryptor REVERSE_ENCRYPTOR = PropertyEncryptorFactory.getPropertyEncryptor(mockProperties([(NiFiProperties.SENSITIVE_PROPS_KEY): REVERSE_KEY]))
-        final String EXPECTED_PLAINTEXT = "my.test.value"
-        final String EXPECTED_CIPHERTEXT = REVERSE_ENCRYPTOR.encrypt(EXPECTED_PLAINTEXT)
-        logger.info("Expected ciphertext: ${EXPECTED_CIPHERTEXT}")
-
-        // Act
-        PopularVoteFlowElection election = electionFactoryBean.object
-        logger.info("Got object: ${election}")
-
-        // Assert
-
-        // Violates LoD but need to evaluate nested encryptor can decrypt
-        def encryptor = election.fingerprintFactory.encryptor
-        String decrypted = encryptor.decrypt(EXPECTED_CIPHERTEXT)
-        logger.info("Decrypted plain text: ${decrypted}")
-        assert decrypted == EXPECTED_PLAINTEXT
-    }
-
-}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/flow/TestPopularVoteFlowElection.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/flow/TestPopularVoteFlowElection.java
index f00a56c..8db6eb9 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/flow/TestPopularVoteFlowElection.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/flow/TestPopularVoteFlowElection.java
@@ -20,15 +20,7 @@ package org.apache.nifi.cluster.coordination.flow;
 import org.apache.nifi.cluster.protocol.DataFlow;
 import org.apache.nifi.cluster.protocol.NodeIdentifier;
 import org.apache.nifi.cluster.protocol.StandardDataFlow;
-import org.apache.nifi.encrypt.PropertyEncryptor;
-import org.apache.nifi.encrypt.SensitiveValueEncoder;
-import org.apache.nifi.fingerprint.FingerprintFactory;
-import org.apache.nifi.nar.ExtensionManager;
-import org.apache.nifi.nar.StandardExtensionDiscoveringManager;
 import org.junit.Test;
-import org.mockito.Mockito;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
 
 import java.io.IOException;
 import java.nio.file.Files;
@@ -45,35 +37,9 @@ import static org.junit.Assert.assertTrue;
 
 public class TestPopularVoteFlowElection {
 
-    private PropertyEncryptor createEncryptor() {
-        return new PropertyEncryptor() {
-            @Override
-            public String encrypt(String property) {
-                return property;
-            }
-
-            @Override
-            public String decrypt(String encryptedProperty) {
-                return encryptedProperty;
-            }
-        };
-    }
-
-    private SensitiveValueEncoder createSensitiveValueEncoder() {
-        return new SensitiveValueEncoder() {
-            @Override
-            public String getEncoded(String encryptedPropertyValue) {
-                return String.format("[MASKED] %s", encryptedPropertyValue);
-            }
-        };
-    }
-
     @Test
     public void testOnlyEmptyFlows() throws IOException {
-        final FingerprintFactory fingerprintFactory = Mockito.mock(FingerprintFactory.class);
-        Mockito.when(fingerprintFactory.createFingerprint(Mockito.any(byte[].class))).thenReturn("fingerprint");
-
-        final PopularVoteFlowElection election = new PopularVoteFlowElection(1, TimeUnit.MINUTES, 3, fingerprintFactory);
+        final PopularVoteFlowElection election = new PopularVoteFlowElection(1, TimeUnit.MINUTES, 3);
         final byte[] flow = Files.readAllBytes(Paths.get("src/test/resources/conf/empty-flow.xml"));
 
         assertFalse(election.isElectionComplete());
@@ -95,20 +61,7 @@ public class TestPopularVoteFlowElection {
 
     @Test
     public void testDifferentEmptyFlows() throws IOException {
-        final FingerprintFactory fingerprintFactory = Mockito.mock(FingerprintFactory.class);
-        Mockito.when(fingerprintFactory.createFingerprint(Mockito.any(byte[].class))).thenAnswer(new Answer<String>() {
-            @Override
-            public String answer(final InvocationOnMock invocation) throws Throwable {
-                final byte[] flow = invocation.getArgument(0);
-                final String xml = new String(flow);
-
-                // Return the ID of the root group as the fingerprint.
-                final String fingerprint = xml.replaceAll("(?s:(.*<id>)(.*?)(</id>.*))", "$2");
-                return fingerprint;
-            }
-        });
-
-        final PopularVoteFlowElection election = new PopularVoteFlowElection(1, TimeUnit.MINUTES, 3, fingerprintFactory);
+        final PopularVoteFlowElection election = new PopularVoteFlowElection(1, TimeUnit.MINUTES, 3);
         final byte[] flow1 = Files.readAllBytes(Paths.get("src/test/resources/conf/empty-flow.xml"));
         final byte[] flow2 = Files.readAllBytes(Paths.get("src/test/resources/conf/different-empty-flow.xml"));
 
@@ -133,10 +86,7 @@ public class TestPopularVoteFlowElection {
 
     @Test
     public void testEmptyFlowIgnoredIfNonEmptyFlowExists() throws IOException {
-        final FingerprintFactory fingerprintFactory = Mockito.mock(FingerprintFactory.class);
-        Mockito.when(fingerprintFactory.createFingerprint(Mockito.any(byte[].class))).thenReturn("fingerprint");
-
-        final PopularVoteFlowElection election = new PopularVoteFlowElection(1, TimeUnit.MINUTES, 8, fingerprintFactory);
+        final PopularVoteFlowElection election = new PopularVoteFlowElection(1, TimeUnit.MINUTES, 8);
         final byte[] emptyFlow = Files.readAllBytes(Paths.get("src/test/resources/conf/empty-flow.xml"));
         final byte[] nonEmptyFlow = Files.readAllBytes(Paths.get("src/test/resources/conf/non-empty-flow.xml"));
 
@@ -163,9 +113,7 @@ public class TestPopularVoteFlowElection {
 
     @Test
     public void testAutoGeneratedVsPopulatedFlowElection() throws IOException {
-        final ExtensionManager extensionManager = new StandardExtensionDiscoveringManager();
-        final FingerprintFactory fingerprintFactory = new FingerprintFactory(createEncryptor(), extensionManager, createSensitiveValueEncoder());
-        final PopularVoteFlowElection election = new PopularVoteFlowElection(1, TimeUnit.MINUTES, 4, fingerprintFactory);
+        final PopularVoteFlowElection election = new PopularVoteFlowElection(1, TimeUnit.MINUTES, 4);
         final byte[] emptyFlow = Files.readAllBytes(Paths.get("src/test/resources/conf/auto-generated-empty-flow.xml"));
         final byte[] nonEmptyFlow = Files.readAllBytes(Paths.get("src/test/resources/conf/reporting-task-flow.xml"));
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/http/replication/TestThreadPoolRequestReplicator.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/http/replication/TestThreadPoolRequestReplicator.java
index ba6636f..c79dad3 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/http/replication/TestThreadPoolRequestReplicator.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/http/replication/TestThreadPoolRequestReplicator.java
@@ -26,8 +26,6 @@ import org.apache.nifi.cluster.coordination.http.replication.util.MockReplicatio
 import org.apache.nifi.cluster.coordination.node.NodeConnectionState;
 import org.apache.nifi.cluster.coordination.node.NodeConnectionStatus;
 import org.apache.nifi.cluster.manager.NodeResponse;
-import org.apache.nifi.cluster.manager.exception.ConnectingNodeMutableRequestException;
-import org.apache.nifi.cluster.manager.exception.DisconnectedNodeMutableRequestException;
 import org.apache.nifi.cluster.manager.exception.IllegalClusterStateException;
 import org.apache.nifi.cluster.protocol.NodeIdentifier;
 import org.apache.nifi.events.EventReporter;
@@ -47,19 +45,15 @@ import org.springframework.security.core.context.SecurityContextHolder;
 
 import javax.ws.rs.HttpMethod;
 import javax.ws.rs.ProcessingException;
-import javax.ws.rs.core.MultivaluedHashMap;
 import javax.ws.rs.core.Response;
 import javax.ws.rs.core.Response.Status;
 import java.net.SocketTimeoutException;
 import java.net.URI;
-import java.net.URISyntaxException;
-import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.LinkedHashSet;
-import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.CountDownLatch;
@@ -323,77 +317,6 @@ public class TestThreadPoolRequestReplicator {
         return coordinator;
     }
 
-    @Test
-    public void testMutableRequestRequiresAllNodesConnected() throws URISyntaxException {
-        final ClusterCoordinator coordinator = createClusterCoordinator();
-
-        // build a map of connection state to node ids
-        final Map<NodeConnectionState, List<NodeIdentifier>> nodeMap = new HashMap<>();
-        final List<NodeIdentifier> connectedNodes = new ArrayList<>();
-        connectedNodes.add(new NodeIdentifier("1", "localhost", 8100, "localhost", 8101, "localhost", 8102, 8103, false));
-        connectedNodes.add(new NodeIdentifier("2", "localhost", 8200, "localhost", 8201, "localhost", 8202, 8203, false));
-        nodeMap.put(NodeConnectionState.CONNECTED, connectedNodes);
-
-        final List<NodeIdentifier> otherState = new ArrayList<>();
-        otherState.add(new NodeIdentifier("3", "localhost", 8300, "localhost", 8301, "localhost", 8302, 8303, false));
-        nodeMap.put(NodeConnectionState.CONNECTING, otherState);
-
-        when(coordinator.getConnectionStates()).thenReturn(nodeMap);
-        final NiFiProperties props = NiFiProperties.createBasicNiFiProperties(null);
-
-        final MockReplicationClient client = new MockReplicationClient();
-        final RequestCompletionCallback requestCompletionCallback = (uri, method, responses) -> {
-        };
-
-        final ThreadPoolRequestReplicator replicator = new ThreadPoolRequestReplicator(5, 100, client, coordinator, requestCompletionCallback, EventReporter.NO_OP, props) {
-            @Override
-            public AsyncClusterResponse replicate(Set<NodeIdentifier> nodeIds, String method, URI uri, Object entity, Map<String, String> headers,
-                                                  boolean indicateReplicated, boolean verify) {
-                return null;
-            }
-        };
-
-        try {
-            // set the user
-            final Authentication authentication = new NiFiAuthenticationToken(new NiFiUserDetails(StandardNiFiUser.ANONYMOUS));
-            SecurityContextHolder.getContext().setAuthentication(authentication);
-
-            try {
-                replicator.replicate(HttpMethod.POST, new URI("http://localhost:80/processors/1"), new ProcessorEntity(), new HashMap<>());
-                Assert.fail("Expected ConnectingNodeMutableRequestException");
-            } catch (final ConnectingNodeMutableRequestException e) {
-                // expected behavior
-            }
-
-            nodeMap.remove(NodeConnectionState.CONNECTING);
-            nodeMap.put(NodeConnectionState.DISCONNECTED, otherState);
-            try {
-                replicator.replicate(HttpMethod.POST, new URI("http://localhost:80/processors/1"), new ProcessorEntity(), new HashMap<>());
-                Assert.fail("Expected DisconnectedNodeMutableRequestException");
-            } catch (final DisconnectedNodeMutableRequestException e) {
-                // expected behavior
-            }
-
-            nodeMap.remove(NodeConnectionState.DISCONNECTED);
-            nodeMap.put(NodeConnectionState.DISCONNECTING, otherState);
-            try {
-                replicator.replicate(HttpMethod.POST, new URI("http://localhost:80/processors/1"), new ProcessorEntity(), new HashMap<>());
-                Assert.fail("Expected DisconnectedNodeMutableRequestException");
-            } catch (final DisconnectedNodeMutableRequestException e) {
-                // expected behavior
-            }
-
-            // should not throw an Exception because it's a GET
-            replicator.replicate(HttpMethod.GET, new URI("http://localhost:80/processors/1"), new MultivaluedHashMap<>(), new HashMap<>());
-
-            // should not throw an Exception because all nodes are now connected
-            nodeMap.remove(NodeConnectionState.DISCONNECTING);
-            replicator.replicate(HttpMethod.POST, new URI("http://localhost:80/processors/1"), new ProcessorEntity(), new HashMap<>());
-        } finally {
-            replicator.shutdown();
-        }
-    }
-
     @Test(timeout = 15000)
     public void testOneNodeRejectsTwoPhaseCommit() {
         final Set<NodeIdentifier> nodeIds = new HashSet<>();
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/integration/Cluster.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/integration/Cluster.java
index 7260927..597a3a6 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/integration/Cluster.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/integration/Cluster.java
@@ -27,11 +27,6 @@ import org.apache.nifi.cluster.coordination.flow.PopularVoteFlowElection;
 import org.apache.nifi.cluster.coordination.node.ClusterRoles;
 import org.apache.nifi.controller.status.history.StatusHistoryRepository;
 import org.apache.nifi.controller.status.history.VolatileComponentStatusRepository;
-import org.apache.nifi.encrypt.PropertyEncryptor;
-import org.apache.nifi.encrypt.PropertyEncryptorFactory;
-import org.apache.nifi.encrypt.SensitiveValueEncoder;
-import org.apache.nifi.encrypt.StandardSensitiveValueEncoder;
-import org.apache.nifi.fingerprint.FingerprintFactory;
 import org.apache.nifi.nar.ExtensionDiscoveringManager;
 import org.apache.nifi.nar.StandardExtensionDiscoveringManager;
 import org.apache.nifi.util.NiFiProperties;
@@ -138,11 +133,8 @@ public class Cluster {
 
         final NiFiProperties nifiProperties = NiFiProperties.createBasicNiFiProperties("src/test/resources/conf/nifi.properties", addProps);
 
-        final PropertyEncryptor encryptor = PropertyEncryptorFactory.getPropertyEncryptor(nifiProperties);
-        final SensitiveValueEncoder sensitiveValueEncoder = new StandardSensitiveValueEncoder(nifiProperties);
         final ExtensionDiscoveringManager extensionManager = new StandardExtensionDiscoveringManager();
-        final FingerprintFactory fingerprintFactory = new FingerprintFactory(encryptor, extensionManager, sensitiveValueEncoder);
-        final FlowElection flowElection = new PopularVoteFlowElection(flowElectionTimeoutMillis, TimeUnit.MILLISECONDS, flowElectionMaxNodes, fingerprintFactory);
+        final FlowElection flowElection = new PopularVoteFlowElection(flowElectionTimeoutMillis, TimeUnit.MILLISECONDS, flowElectionMaxNodes);
         final StatusHistoryRepository statusHistoryRepository = new VolatileComponentStatusRepository(nifiProperties);
         final Node node = new Node(nifiProperties, extensionManager, flowElection, statusHistoryRepository);
         node.start();
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/resources/conf/non-empty-flow.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/resources/conf/non-empty-flow.xml
index edb06b4..ed5451d 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/resources/conf/non-empty-flow.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/resources/conf/non-empty-flow.xml
@@ -21,5 +21,6 @@
     <name>Integration Test Flow</name>
     <position x="0.0" y="0.0"/>
     <comment/>
+    <parameterContextId>1234</parameterContextId>
   </rootGroup>
 </flowController>
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java
index 4859c08..b0a3229 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java
@@ -1509,9 +1509,8 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable
         if (starting) { // will ensure that the Processor represented by this node can only be started once
             initiateStart(taskScheduler, administrativeYieldMillis, timeoutMillis, processContextFactory, schedulingAgentCallback);
         } else {
-            final String procName = processorRef.get().toString();
-            LOG.warn("Cannot start {} because it is not currently stopped. Current state is {}", procName, currentState);
-            procLog.warn("Cannot start {} because it is not currently stopped. Current state is {}", new Object[]{procName, currentState});
+            final String procName = processorRef.get().getProcessor().toString();
+            procLog.warn("Cannot start {} because it is not currently stopped. Current state is {}", procName, currentState);
         }
     }
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/flow/AbstractFlowManager.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/flow/AbstractFlowManager.java
index 7c0343d..209596c 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/flow/AbstractFlowManager.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/flow/AbstractFlowManager.java
@@ -46,7 +46,6 @@ import org.apache.nifi.registry.flow.FlowRegistryClient;
 import org.apache.nifi.remote.PublicPort;
 import org.apache.nifi.remote.RemoteGroupPort;
 import org.apache.nifi.util.ReflectionUtils;
-import org.apache.nifi.web.api.entity.ParameterContextReferenceEntity;
 
 import java.util.ArrayList;
 import java.util.Collections;
@@ -59,6 +58,8 @@ import java.util.UUID;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.function.BooleanSupplier;
+import java.util.function.Predicate;
+import java.util.stream.Collectors;
 
 import static java.util.Objects.requireNonNull;
 
@@ -115,6 +116,12 @@ public abstract class AbstractFlowManager implements FlowManager {
         allProcessors.remove(identifier);
     }
 
+    public Set<ProcessorNode> findAllProcessors(final Predicate<ProcessorNode> filter) {
+        return allProcessors.values().stream()
+            .filter(filter)
+            .collect(Collectors.toSet());
+    }
+
     public Connectable findConnectable(final String id) {
         final ProcessorNode procNode = getProcessorNode(id);
         if (procNode != null) {
@@ -421,7 +428,7 @@ public abstract class AbstractFlowManager implements FlowManager {
 
     @Override
     public ParameterContext createParameterContext(final String id, final String name, final Map<String, Parameter> parameters,
-                                                   final List<ParameterContextReferenceEntity> parameterContexts) {
+                                                   final List<String> inheritedContextIds) {
         final boolean namingConflict = parameterContextManager.getParameterContexts().stream()
             .anyMatch(paramContext -> paramContext.getName().equals(name));
 
@@ -433,13 +440,13 @@ public abstract class AbstractFlowManager implements FlowManager {
         final ParameterContext parameterContext = new StandardParameterContext(id, name, referenceManager, getParameterContextParent());
         parameterContext.setParameters(parameters);
 
-        if (parameterContexts != null && !parameterContexts.isEmpty()) {
+        if (inheritedContextIds != null && !inheritedContextIds.isEmpty()) {
             if (!withParameterContextResolution.get()) {
                 throw new IllegalStateException("A ParameterContext with inherited ParameterContexts may only be created from within a call to AbstractFlowManager#withParameterContextResolution");
             }
             final List<ParameterContext> parameterContextList = new ArrayList<>();
-            for(final ParameterContextReferenceEntity parameterContextRef : parameterContexts) {
-                parameterContextList.add(lookupParameterContext(parameterContextRef.getId()));
+            for(final String inheritedContextId : inheritedContextIds) {
+                parameterContextList.add(lookupParameterContext(inheritedContextId));
             }
             parameterContext.setInheritedParameterContexts(parameterContextList);
         }
@@ -451,8 +458,11 @@ public abstract class AbstractFlowManager implements FlowManager {
     @Override
     public void withParameterContextResolution(final Runnable parameterContextAction) {
         withParameterContextResolution.set(true);
-        parameterContextAction.run();
-        withParameterContextResolution.set(false);
+        try {
+            parameterContextAction.run();
+        } finally {
+            withParameterContextResolution.set(false);
+        }
 
         for (final ParameterContext parameterContext : parameterContextManager.getParameterContexts()) {
             // if a param context in the manager itself is reference-only, it means there is a reference to a param
@@ -495,8 +505,8 @@ public abstract class AbstractFlowManager implements FlowManager {
         if (!parameterContextManager.hasParameterContext(id)) {
             parameterContextManager.addParameterContext(new ReferenceOnlyParameterContext(id));
         }
-        return parameterContextManager.getParameterContext(id);
 
+        return parameterContextManager.getParameterContext(id);
     }
 
     protected abstract Authorizable getParameterContextParent();
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/reporting/AbstractReportingTaskNode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/reporting/AbstractReportingTaskNode.java
index 019720e..63cde33 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/reporting/AbstractReportingTaskNode.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/reporting/AbstractReportingTaskNode.java
@@ -23,6 +23,7 @@ import org.apache.nifi.components.ConfigVerificationResult;
 import org.apache.nifi.components.ConfigVerificationResult.Outcome;
 import org.apache.nifi.components.ConfigurableComponent;
 import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.components.validation.ValidationState;
 import org.apache.nifi.components.validation.ValidationStatus;
 import org.apache.nifi.components.validation.ValidationTrigger;
 import org.apache.nifi.controller.AbstractComponentNode;
@@ -258,12 +259,10 @@ public abstract class AbstractReportingTaskNode extends AbstractComponentNode im
             throw new IllegalStateException("Cannot start " + getReportingTask().getIdentifier() + " because it is currently disabled");
         }
 
-        if (isRunning()) {
-            throw new IllegalStateException("Cannot start " + getReportingTask().getIdentifier() + " because it is already running");
-        }
-
-        if (getValidationStatus() == ValidationStatus.INVALID) {
-            throw new IllegalStateException("Cannot start " + getReportingTask().getIdentifier() + " because it is in INVALID status");
+        final ValidationState validationState = getValidationState();
+        if (validationState.getStatus() == ValidationStatus.INVALID) {
+            throw new IllegalStateException("Cannot start " + getReportingTask().getIdentifier() +
+                " because it is invalid with the following validation errors: " + validationState.getValidationErrors());
         }
     }
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java
index baccdfd..a8e032a 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java
@@ -174,6 +174,11 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
 
     @Override
     public CompletableFuture<Void> enableControllerService(final ControllerServiceNode serviceNode) {
+        if (serviceNode.isActive()) {
+            final CompletableFuture<Void> future = CompletableFuture.completedFuture(null);
+            return future;
+        }
+
         serviceNode.verifyCanEnable();
         serviceNode.reloadAdditionalResourcesIfNecessary();
         return processScheduler.enableControllerService(serviceNode);
@@ -231,18 +236,6 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
     private void enableControllerServices(final Collection<ControllerServiceNode> serviceNodes, final CompletableFuture<Void> completableFuture) {
         // validate that we are able to start all of the services.
         for (final ControllerServiceNode controllerServiceNode : serviceNodes) {
-            List<ControllerServiceNode> requiredServices = controllerServiceNode.getRequiredControllerServices();
-            for (ControllerServiceNode requiredService : requiredServices) {
-                if (!requiredService.isActive() && !serviceNodes.contains(requiredService)) {
-                    logger.error("Cannot enable {} because it has a dependency on {}, which is not enabled", controllerServiceNode, requiredService);
-                    completableFuture.completeExceptionally(new IllegalStateException("Cannot enable " + controllerServiceNode
-                        + " because it has a dependency on " + requiredService + ", which is not enabled"));
-                    return;
-                }
-            }
-        }
-
-        for (final ControllerServiceNode controllerServiceNode : serviceNodes) {
             if (completableFuture.isCancelled()) {
                 return;
             }
@@ -285,8 +278,7 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
 
     @Override
     public Future<Void> enableControllerServiceAndDependencies(final ControllerServiceNode serviceNode) {
-        final ControllerServiceState currentState = serviceNode.getState();
-        if (currentState == ControllerServiceState.ENABLED) {
+        if (serviceNode.isActive()) {
             logger.debug("Enabling of Controller Service {} triggered but service already enabled", serviceNode);
             return CompletableFuture.completedFuture(null);
         }
@@ -533,10 +525,10 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
         final ProcessGroup group = serviceNode.getProcessGroup();
         if (group == null) {
             flowManager.removeRootControllerService(serviceNode);
-            return;
+        } else {
+            group.removeControllerService(serviceNode);
         }
 
-        group.removeControllerService(serviceNode);
         LogRepositoryFactory.removeRepository(serviceNode.getIdentifier());
         extensionManager.removeInstanceClassLoader(serviceNode.getIdentifier());
         serviceCache.remove(serviceNode.getIdentifier());
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/groups/DefaultComponentScheduler.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/groups/DefaultComponentScheduler.java
new file mode 100644
index 0000000..ee7fa9b
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/groups/DefaultComponentScheduler.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.groups;
+
+import org.apache.nifi.connectable.Connectable;
+import org.apache.nifi.connectable.Port;
+import org.apache.nifi.controller.ProcessorNode;
+import org.apache.nifi.remote.RemoteGroupPort;
+
+public class DefaultComponentScheduler extends AbstractComponentScheduler {
+    @Override
+    protected void startNow(final Connectable component) {
+        switch (component.getConnectableType()) {
+            case PROCESSOR: {
+                final ProcessorNode processorNode = (ProcessorNode) component;
+                processorNode.getProcessGroup().startProcessor(processorNode, false);
+                break;
+            }
+            case INPUT_PORT: {
+                final Port port = (Port) component;
+                port.getProcessGroup().startInputPort(port);
+                break;
+            }
+            case OUTPUT_PORT: {
+                final Port port = (Port) component;
+                port.getProcessGroup().startOutputPort(port);
+                break;
+            }
+            case REMOTE_INPUT_PORT:
+            case REMOTE_OUTPUT_PORT: {
+                final RemoteGroupPort port = (RemoteGroupPort) component;
+                port.getRemoteProcessGroup().startTransmitting(port);
+            }
+        }
+    }
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/groups/ProcessGroupSynchronizationContext.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/groups/ProcessGroupSynchronizationContext.java
new file mode 100644
index 0000000..4e18031
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/groups/ProcessGroupSynchronizationContext.java
@@ -0,0 +1,163 @@
+/*
+ * 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.groups;
+
+import org.apache.nifi.controller.ProcessorNode;
+import org.apache.nifi.controller.ReloadComponent;
+import org.apache.nifi.controller.flow.FlowManager;
+import org.apache.nifi.controller.service.ControllerServiceProvider;
+import org.apache.nifi.nar.ExtensionManager;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.registry.flow.FlowRegistryClient;
+import org.apache.nifi.registry.flow.mapping.FlowMappingOptions;
+
+import java.util.function.Function;
+
+import static java.util.Objects.requireNonNull;
+
+public class ProcessGroupSynchronizationContext {
+    private final ComponentIdGenerator componentIdGenerator;
+    private final FlowManager flowManager;
+    private final FlowRegistryClient flowRegistryClient;
+    private final ReloadComponent reloadComponent;
+    private final ControllerServiceProvider controllerServiceProvider;
+    private final ExtensionManager extensionManager;
+    private final ComponentScheduler componentScheduler;
+    private final FlowMappingOptions flowMappingOptions;
+    private final Function<ProcessorNode, ProcessContext> processContextFactory;
+
+
+    private ProcessGroupSynchronizationContext(final Builder builder) {
+        this.componentIdGenerator = builder.componentIdGenerator;
+        this.flowManager = builder.flowManager;
+        this.flowRegistryClient = builder.flowRegistryClient;
+        this.reloadComponent = builder.reloadComponent;
+        this.controllerServiceProvider = builder.controllerServiceProvider;
+        this.extensionManager = builder.extensionManager;
+        this.componentScheduler = builder.componentScheduler;
+        this.flowMappingOptions = builder.flowMappingOptions;
+        this.processContextFactory = builder.processContextFactory;
+    }
+
+    public ComponentIdGenerator getComponentIdGenerator() {
+        return componentIdGenerator;
+    }
+
+    public FlowManager getFlowManager() {
+        return flowManager;
+    }
+
+    public FlowRegistryClient getFlowRegistryClient() {
+        return flowRegistryClient;
+    }
+
+    public ReloadComponent getReloadComponent() {
+        return reloadComponent;
+    }
+
+    public ControllerServiceProvider getControllerServiceProvider() {
+        return controllerServiceProvider;
+    }
+
+    public ExtensionManager getExtensionManager() {
+        return extensionManager;
+    }
+
+    public ComponentScheduler getComponentScheduler() {
+        return componentScheduler;
+    }
+
+    public FlowMappingOptions getFlowMappingOptions() {
+        return flowMappingOptions;
+    }
+
+    public Function<ProcessorNode, ProcessContext> getProcessContextFactory() {
+        return processContextFactory;
+    }
+
+    public static class Builder {
+        private ComponentIdGenerator componentIdGenerator;
+        private FlowManager flowManager;
+        private FlowRegistryClient flowRegistryClient;
+        private ReloadComponent reloadComponent;
+        private ControllerServiceProvider controllerServiceProvider;
+        private ExtensionManager extensionManager;
+        private ComponentScheduler componentScheduler;
+        private FlowMappingOptions flowMappingOptions;
+        private Function<ProcessorNode, ProcessContext> processContextFactory;
+
+        public Builder componentIdGenerator(final ComponentIdGenerator componentIdGenerator) {
+            this.componentIdGenerator = componentIdGenerator;
+            return this;
+        }
+
+        public Builder flowManager(final FlowManager flowManager) {
+            this.flowManager = flowManager;
+            return this;
+        }
+
+        public Builder flowRegistryClient(final FlowRegistryClient client) {
+            this.flowRegistryClient = client;
+            return this;
+        }
+
+        public Builder reloadComponent(final ReloadComponent reloadComponent) {
+            this.reloadComponent = reloadComponent;
+            return this;
+        }
+
+        public Builder controllerServiceProvider(final ControllerServiceProvider provider) {
+            this.controllerServiceProvider = provider;
+            return this;
+        }
+
+        public Builder extensionManager(final ExtensionManager extensionManager) {
+            this.extensionManager = extensionManager;
+            return this;
+        }
+
+        public Builder componentScheduler(final ComponentScheduler scheduler) {
+            this.componentScheduler = scheduler;
+            return this;
+        }
+
+        public Builder flowMappingOptions(final FlowMappingOptions flowMappingOptions) {
+            this.flowMappingOptions = flowMappingOptions;
+            return this;
+        }
+
+        public Builder processContextFactory(final Function<ProcessorNode, ProcessContext> processContextFactory) {
+            this.processContextFactory = processContextFactory;
+            return this;
+        }
+
+        public ProcessGroupSynchronizationContext build() {
+            requireNonNull(componentIdGenerator, "Component ID Generator must be set");
+            requireNonNull(flowManager, "Flow Manager must be set");
+            requireNonNull(flowRegistryClient, "Flow Registry Client must be set");
+            requireNonNull(reloadComponent, "Reload Component must be set");
+            requireNonNull(controllerServiceProvider, "Controller Service Provider must be set");
+            requireNonNull(extensionManager, "Extension Manager must be set");
+            requireNonNull(componentScheduler, "Component Scheduler must be set");
+            requireNonNull(flowMappingOptions, "Flow Mapping Options must be set");
+            requireNonNull(processContextFactory, "Process Context Factory must be set");
+            return new ProcessGroupSynchronizationContext(this);
+        }
+    }
+
+}
diff --git a/nifi-registry/nifi-registry-core/nifi-registry-flow-diff/src/main/java/org/apache/nifi/registry/flow/diff/ComparableDataFlow.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/groups/ProcessGroupSynchronizer.java
similarity index 63%
copy from nifi-registry/nifi-registry-core/nifi-registry-flow-diff/src/main/java/org/apache/nifi/registry/flow/diff/ComparableDataFlow.java
copy to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/groups/ProcessGroupSynchronizer.java
index a9ceca5..4f7a309 100644
--- a/nifi-registry/nifi-registry-core/nifi-registry-flow-diff/src/main/java/org/apache/nifi/registry/flow/diff/ComparableDataFlow.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/groups/ProcessGroupSynchronizer.java
@@ -15,12 +15,16 @@
  * limitations under the License.
  */
 
-package org.apache.nifi.registry.flow.diff;
+package org.apache.nifi.groups;
 
+import org.apache.nifi.controller.exception.ProcessorInstantiationException;
 import org.apache.nifi.flow.VersionedProcessGroup;
+import org.apache.nifi.registry.flow.VersionedFlowSnapshot;
 
-public interface ComparableDataFlow {
-    String getName();
+public interface ProcessGroupSynchronizer {
+
+    void synchronize(ProcessGroup group, VersionedFlowSnapshot proposedSnapshot, GroupSynchronizationOptions synchronizationOptions) throws ProcessorInstantiationException;
+
+    void verifyCanSynchronize(ProcessGroup group, VersionedProcessGroup proposed, boolean verifyConnectionRemoval);
 
-    VersionedProcessGroup getContents();
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java
index 670db60..3e679c7 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java
@@ -26,7 +26,6 @@ import org.apache.nifi.authorization.Resource;
 import org.apache.nifi.authorization.resource.Authorizable;
 import org.apache.nifi.authorization.resource.ResourceFactory;
 import org.apache.nifi.authorization.resource.ResourceType;
-import org.apache.nifi.bundle.BundleCoordinate;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.state.StateManager;
 import org.apache.nifi.components.state.StateManagerProvider;
@@ -39,7 +38,6 @@ import org.apache.nifi.connectable.LocalPort;
 import org.apache.nifi.connectable.Port;
 import org.apache.nifi.connectable.Position;
 import org.apache.nifi.connectable.Positionable;
-import org.apache.nifi.connectable.Size;
 import org.apache.nifi.controller.ComponentNode;
 import org.apache.nifi.controller.ConfigurationContext;
 import org.apache.nifi.controller.ControllerService;
@@ -53,15 +51,12 @@ import org.apache.nifi.controller.ScheduledState;
 import org.apache.nifi.controller.Snippet;
 import org.apache.nifi.controller.Template;
 import org.apache.nifi.controller.exception.ComponentLifeCycleException;
-import org.apache.nifi.controller.exception.ProcessorInstantiationException;
 import org.apache.nifi.controller.flow.FlowManager;
 import org.apache.nifi.controller.label.Label;
 import org.apache.nifi.controller.queue.DropFlowFileRequest;
 import org.apache.nifi.controller.queue.DropFlowFileState;
 import org.apache.nifi.controller.queue.DropFlowFileStatus;
 import org.apache.nifi.controller.queue.FlowFileQueue;
-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.controller.service.ControllerServiceNode;
 import org.apache.nifi.controller.service.ControllerServiceProvider;
@@ -69,24 +64,7 @@ import org.apache.nifi.controller.service.ControllerServiceReference;
 import org.apache.nifi.controller.service.ControllerServiceState;
 import org.apache.nifi.controller.service.StandardConfigurationContext;
 import org.apache.nifi.encrypt.PropertyEncryptor;
-import org.apache.nifi.flow.BatchSize;
-import org.apache.nifi.flow.Bundle;
-import org.apache.nifi.flow.ComponentType;
-import org.apache.nifi.flow.ConnectableComponent;
-import org.apache.nifi.flow.VersionedComponent;
-import org.apache.nifi.flow.VersionedConnection;
-import org.apache.nifi.flow.VersionedControllerService;
-import org.apache.nifi.flow.VersionedFlowCoordinates;
-import org.apache.nifi.flow.VersionedFunnel;
-import org.apache.nifi.flow.VersionedLabel;
-import org.apache.nifi.flow.VersionedPort;
 import org.apache.nifi.flow.VersionedProcessGroup;
-import org.apache.nifi.flow.VersionedProcessor;
-import org.apache.nifi.flow.VersionedPropertyDescriptor;
-import org.apache.nifi.flow.VersionedRemoteGroupPort;
-import org.apache.nifi.flow.VersionedRemoteProcessGroup;
-import org.apache.nifi.flowfile.FlowFilePrioritizer;
-import org.apache.nifi.logging.LogLevel;
 import org.apache.nifi.logging.LogRepository;
 import org.apache.nifi.logging.LogRepositoryFactory;
 import org.apache.nifi.nar.ExtensionManager;
@@ -98,7 +76,7 @@ import org.apache.nifi.parameter.ParameterReference;
 import org.apache.nifi.parameter.ParameterUpdate;
 import org.apache.nifi.parameter.StandardParameterUpdate;
 import org.apache.nifi.processor.DataUnit;
-import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.StandardProcessContext;
 import org.apache.nifi.registry.ComponentVariableRegistry;
 import org.apache.nifi.registry.VariableDescriptor;
@@ -111,43 +89,33 @@ import org.apache.nifi.registry.flow.VersionedFlow;
 import org.apache.nifi.registry.flow.VersionedFlowSnapshot;
 import org.apache.nifi.registry.flow.VersionedFlowState;
 import org.apache.nifi.registry.flow.VersionedFlowStatus;
-import org.apache.nifi.registry.flow.VersionedParameter;
-import org.apache.nifi.registry.flow.VersionedParameterContext;
 import org.apache.nifi.registry.flow.diff.ComparableDataFlow;
-import org.apache.nifi.registry.flow.diff.DifferenceType;
 import org.apache.nifi.registry.flow.diff.EvolvingDifferenceDescriptor;
 import org.apache.nifi.registry.flow.diff.FlowComparator;
 import org.apache.nifi.registry.flow.diff.FlowComparison;
 import org.apache.nifi.registry.flow.diff.FlowDifference;
 import org.apache.nifi.registry.flow.diff.StandardComparableDataFlow;
 import org.apache.nifi.registry.flow.diff.StandardFlowComparator;
-import org.apache.nifi.registry.flow.diff.StaticDifferenceDescriptor;
+import org.apache.nifi.registry.flow.mapping.ComponentIdLookup;
+import org.apache.nifi.registry.flow.mapping.FlowMappingOptions;
 import org.apache.nifi.registry.flow.mapping.NiFiRegistryFlowMapper;
+import org.apache.nifi.registry.flow.mapping.VersionedComponentStateLookup;
 import org.apache.nifi.registry.variable.MutableVariableRegistry;
 import org.apache.nifi.remote.PublicPort;
 import org.apache.nifi.remote.RemoteGroupPort;
-import org.apache.nifi.remote.StandardRemoteProcessGroupPortDescriptor;
-import org.apache.nifi.remote.TransferDirection;
-import org.apache.nifi.remote.protocol.SiteToSiteTransportProtocol;
-import org.apache.nifi.scheduling.ExecutionNode;
-import org.apache.nifi.scheduling.SchedulingStrategy;
 import org.apache.nifi.util.FlowDifferenceFilters;
 import org.apache.nifi.util.FormatUtils;
 import org.apache.nifi.util.NiFiProperties;
 import org.apache.nifi.util.ReflectionUtils;
 import org.apache.nifi.util.SnippetUtils;
-import org.apache.nifi.web.ResourceNotFoundException;
 import org.apache.nifi.web.Revision;
-import org.apache.nifi.web.api.dto.ParameterContextReferenceDTO;
 import org.apache.nifi.web.api.dto.TemplateDTO;
 import org.apache.nifi.web.api.dto.VersionedFlowDTO;
-import org.apache.nifi.web.api.entity.ParameterContextReferenceEntity;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 import java.net.ConnectException;
-import java.net.URL;
 import java.nio.charset.StandardCharsets;
 import java.security.SecureRandom;
 import java.util.ArrayList;
@@ -156,7 +124,6 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
@@ -166,7 +133,6 @@ import java.util.UUID;
 import java.util.concurrent.Callable;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.Future;
-import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.concurrent.locks.Lock;
@@ -179,6 +145,7 @@ import java.util.stream.Collectors;
 
 import static java.util.Objects.requireNonNull;
 
+
 public final class StandardProcessGroup implements ProcessGroup {
     public static final List<DropFlowFileState> AGGREGATE_DROP_FLOW_FILE_STATE_PRECEDENCES = Arrays.asList(
         DropFlowFileState.FAILURE,
@@ -278,7 +245,7 @@ public final class StandardProcessGroup implements ProcessGroup {
             nifiPropertiesBackpressureSize = DEFAULT_BACKPRESSURE_DATA_SIZE;
         } else {
             // Validate the property values.
-            Long count;
+            long count;
             try {
                 final String explicitValue = nifiProperties.getProperty(NiFiProperties.BACKPRESSURE_COUNT, String.valueOf(DEFAULT_BACKPRESSURE_OBJECT));
                 count = Long.parseLong(explicitValue);
@@ -549,9 +516,7 @@ public final class StandardProcessGroup implements ProcessGroup {
 
             findAllInputPorts().stream().filter(START_PORTS_FILTER).forEach(port -> port.getProcessGroup().startInputPort(port));
 
-            findAllOutputPorts().stream().filter(START_PORTS_FILTER).forEach(port -> {
-                port.getProcessGroup().startOutputPort(port);
-            });
+            findAllOutputPorts().stream().filter(START_PORTS_FILTER).forEach(port -> port.getProcessGroup().startOutputPort(port));
         } finally {
             readLock.unlock();
         }
@@ -641,7 +606,7 @@ public final class StandardProcessGroup implements ProcessGroup {
         writeLock.lock();
         try {
             // Unique port check within the same group.
-            verifyPortUniqueness(port, inputPorts, name -> getInputPortByName(name));
+            verifyPortUniqueness(port, inputPorts, this::getInputPortByName);
 
             port.setProcessGroup(this);
             inputPorts.put(requireNonNull(port).getIdentifier(), port);
@@ -3635,6 +3600,7 @@ public final class StandardProcessGroup implements ProcessGroup {
         copy.setRemoteProcessGroups(processGroup.getRemoteProcessGroups());
         copy.setVariables(processGroup.getVariables());
         copy.setLabels(processGroup.getLabels());
+        copy.setParameterContextName(processGroup.getParameterContextName());
 
         final Set<VersionedProcessGroup> copyChildren = new HashSet<>();
 
@@ -3655,6 +3621,7 @@ public final class StandardProcessGroup implements ProcessGroup {
                 childCopy.setDefaultFlowFileExpiration(childGroup.getDefaultFlowFileExpiration());
                 childCopy.setDefaultBackPressureObjectThreshold(childGroup.getDefaultBackPressureObjectThreshold());
                 childCopy.setDefaultBackPressureDataSizeThreshold(childGroup.getDefaultBackPressureDataSizeThreshold());
+                childCopy.setParameterContextName(childGroup.getParameterContextName());
 
                 copyChildren.add(childCopy);
             }
@@ -3803,7 +3770,7 @@ public final class StandardProcessGroup implements ProcessGroup {
 
             versionControlFields.setSyncFailureExplanation(null);
         } catch (final IOException | NiFiRegistryException e) {
-            final String message = String.format("Failed to synchronize Process Group with Flow Registry : " + e.getMessage());
+            final String message = "Failed to synchronize Process Group with Flow Registry : " + e.getMessage();
             versionControlFields.setSyncFailureExplanation(message);
 
             LOG.error("Failed to synchronize {} with Flow Registry because could not determine the most recent version of the Flow in the Flow Registry", this, e);
@@ -3813,73 +3780,53 @@ public final class StandardProcessGroup implements ProcessGroup {
     @Override
     public void updateFlow(final VersionedFlowSnapshot proposedSnapshot, final String componentIdSeed, final boolean verifyNotDirty, final boolean updateSettings,
                            final boolean updateDescendantVersionedFlows) {
-        writeLock.lock();
-        try {
-            verifyCanUpdate(proposedSnapshot, true, verifyNotDirty);
-
-            final NiFiRegistryFlowMapper mapper = new NiFiRegistryFlowMapper(extensionManager);
-            final VersionedProcessGroup versionedGroup = mapper.mapProcessGroup(this, controllerServiceProvider, flowRegistryClient, true);
-
-            final ComparableDataFlow localFlow = new StandardComparableDataFlow("Current Flow", versionedGroup);
-            final ComparableDataFlow proposedFlow = new StandardComparableDataFlow("New Flow", proposedSnapshot.getFlowContents());
 
-            final FlowComparator flowComparator = new StandardFlowComparator(proposedFlow, localFlow, getAncestorServiceIds(), new StaticDifferenceDescriptor());
-            final FlowComparison flowComparison = flowComparator.compare();
+        final ComponentIdGenerator idGenerator = (proposedId, instanceId, destinationGroupId) -> generateUuid(proposedId, destinationGroupId, componentIdSeed);
+        final ComponentScheduler componentScheduler = new DefaultComponentScheduler();
 
-            final Set<String> updatedVersionedComponentIds = new HashSet<>();
-            for (final FlowDifference diff : flowComparison.getDifferences()) {
-                // Ignore these as local differences for now because we can't do anything with it
-                if (diff.getDifferenceType() == DifferenceType.BUNDLE_CHANGED) {
-                    continue;
-                }
+        final GroupSynchronizationOptions synchronizationOptions = new GroupSynchronizationOptions.Builder()
+            .componentIdGenerator(idGenerator)
+            .componentScheduler(componentScheduler)
+            .ignoreLocalModifications(!verifyNotDirty)
+            .updateDescendantVersionedFlows(updateDescendantVersionedFlows)
+            .updateGroupSettings(updateSettings)
+            .updateGroupVersionControlSnapshot(true)
+            .updateExistingVariables(false)
+            .updateRpgUrls(false)
+            .propertyDecryptor(value -> null)
+            .build();
 
-                // If this update adds a new Controller Service, then we need to check if the service already exists at a higher level
-                // and if so compare our VersionedControllerService to the existing service.
-                if (diff.getDifferenceType() == DifferenceType.COMPONENT_ADDED) {
-                    final VersionedComponent component = diff.getComponentA() == null ? diff.getComponentB() : diff.getComponentA();
-                    if (ComponentType.CONTROLLER_SERVICE == component.getComponentType()) {
-                        final ControllerServiceNode serviceNode = getVersionedControllerService(this, component.getIdentifier());
-                        if (serviceNode != null) {
-                            final VersionedControllerService versionedService = mapper.mapControllerService(serviceNode, controllerServiceProvider,
-                                Collections.singleton(serviceNode.getProcessGroupIdentifier()), new HashMap<>());
-                            final Set<FlowDifference> differences = flowComparator.compareControllerServices(versionedService, (VersionedControllerService) component);
+        final FlowMappingOptions flowMappingOptions = new FlowMappingOptions.Builder()
+            .mapSensitiveConfiguration(false)
+            .mapPropertyDescriptors(true)
+            .stateLookup(VersionedComponentStateLookup.ENABLED_OR_DISABLED)
+            .sensitiveValueEncryptor(null)
+            .componentIdLookup(ComponentIdLookup.VERSIONED_OR_GENERATE)
+            .mapInstanceIdentifiers(false)
+            .mapControllerServiceReferencesToVersionedId(true)
+            .build();
 
-                            if (!differences.isEmpty()) {
-                                updatedVersionedComponentIds.add(component.getIdentifier());
-                            }
-
-                            continue;
-                        }
-                    }
-                }
-
-                final VersionedComponent component = diff.getComponentA() == null ? diff.getComponentB() : diff.getComponentA();
-                updatedVersionedComponentIds.add(component.getIdentifier());
-
-                if (component.getComponentType() == ComponentType.REMOTE_INPUT_PORT || component.getComponentType() == ComponentType.REMOTE_OUTPUT_PORT) {
-                    final String remoteGroupId = ((VersionedRemoteGroupPort) component).getRemoteGroupId();
-                    updatedVersionedComponentIds.add(remoteGroupId);
-                }
-            }
+        synchronizeFlow(proposedSnapshot, synchronizationOptions, flowMappingOptions);
+    }
 
-            if (LOG.isInfoEnabled()) {
-                final String differencesByLine = flowComparison.getDifferences().stream()
-                    .map(FlowDifference::toString)
-                    .collect(Collectors.joining("\n"));
+    private ProcessContext createProcessContext(final ProcessorNode processorNode) {
+        return new StandardProcessContext(processorNode, controllerServiceProvider, encryptor,
+            stateManagerProvider.getStateManager(processorNode.getIdentifier()), () -> false, nodeTypeProvider);
+    }
 
-                // TODO: Until we move to NiFi Registry 0.6.0, avoid using proposedSnapshot.toString() because it throws a NullPointerException
-                final String proposedSnapshotDetails = "VersionedFlowSnapshot[flowContentsId=" + proposedSnapshot.getFlowContents().getIdentifier()
-                    + ", flowContentsName=" + proposedSnapshot.getFlowContents().getName() + ", NoMetadataAvailable]";
-                LOG.info("Updating {} to {}; there are {} differences to take into account:\n{}", this, proposedSnapshotDetails,
-                    flowComparison.getDifferences().size(), differencesByLine);
-            }
+    @Override
+    public void synchronizeFlow(final VersionedFlowSnapshot proposedSnapshot, final GroupSynchronizationOptions synchronizationOptions, final FlowMappingOptions flowMappingOptions) {
+        writeLock.lock();
+        try {
+            verifyCanUpdate(proposedSnapshot, true, !synchronizationOptions.isIgnoreLocalModifications());
 
-            final Set<String> knownVariables = getKnownVariableNames();
+            final ProcessGroupSynchronizationContext groupSynchronizationContext = createGroupSynchronizationContext(
+                synchronizationOptions.getComponentIdGenerator(), synchronizationOptions.getComponentScheduler(), flowMappingOptions);
+            final StandardProcessGroupSynchronizer synchronizer = new StandardProcessGroupSynchronizer(groupSynchronizationContext);
 
             final StandardVersionControlInformation originalVci = this.versionControlInfo.get();
             try {
-                updateProcessGroup(this, proposedSnapshot.getFlowContents(), componentIdSeed, updatedVersionedComponentIds, false, updateSettings, updateDescendantVersionedFlows, knownVariables,
-                    proposedSnapshot.getParameterContexts());
+                synchronizer.synchronize(this, proposedSnapshot, synchronizationOptions);
             } catch (final Throwable t) {
                 // The proposed snapshot may not have any Versioned Flow Coordinates. As a result, the call to #updateProcessGroup may
                 // set this PG's Version Control Info to null. During the normal flow of control,
@@ -3893,13 +3840,12 @@ public final class StandardProcessGroup implements ProcessGroup {
 
                 throw t;
             }
-        } catch (final ProcessorInstantiationException pie) {
-            throw new IllegalStateException("Failed to update flow", pie);
         } finally {
             writeLock.unlock();
         }
     }
 
+
     @Override
     public Set<String> getAncestorServiceIds() {
         final Set<String> ancestorServiceIds;
@@ -3920,699 +3866,6 @@ public final class StandardProcessGroup implements ProcessGroup {
         return ancestorServiceIds;
     }
 
-    private ControllerServiceNode getVersionedControllerService(final ProcessGroup group, final String versionedComponentId) {
-        if (group == null) {
-            return null;
-        }
-
-        for (final ControllerServiceNode serviceNode : group.getControllerServices(false)) {
-            final String serviceNodeVersionedComponentId = serviceNode.getVersionedComponentId().orElse(
-                NiFiRegistryFlowMapper.generateVersionedComponentId(serviceNode.getIdentifier()));
-            if (serviceNodeVersionedComponentId.equals(versionedComponentId)) {
-                return serviceNode;
-            }
-        }
-
-        return getVersionedControllerService(group.getParent(), versionedComponentId);
-    }
-
-    private Set<String> getKnownVariableNames() {
-        final Set<String> variableNames = new HashSet<>();
-        populateKnownVariableNames(this, variableNames);
-        return variableNames;
-    }
-
-    private void populateKnownVariableNames(final ProcessGroup group, final Set<String> knownVariables) {
-        group.getVariableRegistry().getVariableMap().keySet().stream()
-            .map(VariableDescriptor::getName)
-            .forEach(knownVariables::add);
-
-        final ProcessGroup parent = group.getParent();
-        if (parent != null) {
-            populateKnownVariableNames(parent, knownVariables);
-        }
-    }
-
-    private void updateProcessGroup(final ProcessGroup group, final VersionedProcessGroup proposed, final String componentIdSeed,
-                                    final Set<String> updatedVersionedComponentIds, final boolean updatePosition, final boolean updateName, final boolean updateDescendantVersionedGroups,
-                                    final Set<String> variablesToSkip, final Map<String, VersionedParameterContext> versionedParameterContexts) throws ProcessorInstantiationException {
-
-        // During the flow update, we will use temporary names for process group ports. This is because port names must be
-        // unique within a process group, but during an update we might temporarily be in a state where two ports have the same name.
-        // For example, if a process group update involves removing/renaming port A, and then adding/updating port B where B is given
-        // A's former name. This is a valid state by the end of the flow update, but for a brief moment there may be two ports with the
-        // same name. To avoid this conflict, we keep the final names in a map indexed by port id, use a temporary name for each port
-        // during the update, and after all ports have been added/updated/removed, we set the final names on all ports.
-        final Map<Port, String> proposedPortFinalNames = new HashMap<>();
-
-        group.setComments(proposed.getComments());
-
-        if (updateName) {
-            group.setName(proposed.getName());
-        }
-
-        if (updatePosition && proposed.getPosition() != null) {
-            group.setPosition(new Position(proposed.getPosition().getX(), proposed.getPosition().getY()));
-        }
-
-        flowManager.withParameterContextResolution(() -> updateParameterContext(group, proposed, versionedParameterContexts, componentIdSeed));
-        updateVariableRegistry(group, proposed, variablesToSkip);
-
-        final FlowFileConcurrency flowFileConcurrency = proposed.getFlowFileConcurrency() == null ? FlowFileConcurrency.UNBOUNDED :
-            FlowFileConcurrency.valueOf(proposed.getFlowFileConcurrency());
-        group.setFlowFileConcurrency(flowFileConcurrency);
-
-        final FlowFileOutboundPolicy outboundPolicy = proposed.getFlowFileOutboundPolicy() == null ? FlowFileOutboundPolicy.STREAM_WHEN_AVAILABLE :
-            FlowFileOutboundPolicy.valueOf(proposed.getFlowFileOutboundPolicy());
-        group.setFlowFileOutboundPolicy(outboundPolicy);
-
-        group.setDefaultFlowFileExpiration(proposed.getDefaultFlowFileExpiration());
-        group.setDefaultBackPressureObjectThreshold(proposed.getDefaultBackPressureObjectThreshold());
-        group.setDefaultBackPressureDataSizeThreshold(proposed.getDefaultBackPressureDataSizeThreshold());
-
-        final VersionedFlowCoordinates remoteCoordinates = proposed.getVersionedFlowCoordinates();
-        if (remoteCoordinates == null) {
-            group.disconnectVersionControl(false);
-        } else {
-            final String registryId = flowRegistryClient.getFlowRegistryId(remoteCoordinates.getRegistryUrl());
-            final String bucketId = remoteCoordinates.getBucketId();
-            final String flowId = remoteCoordinates.getFlowId();
-            final int version = remoteCoordinates.getVersion();
-
-            final FlowRegistry flowRegistry = flowRegistryClient.getFlowRegistry(registryId);
-            final String registryName = flowRegistry == null ? registryId : flowRegistry.getName();
-
-            final VersionedFlowState flowState = remoteCoordinates.getLatest() ? VersionedFlowState.UP_TO_DATE : VersionedFlowState.STALE;
-
-            final VersionControlInformation vci = new StandardVersionControlInformation.Builder()
-                .registryId(registryId)
-                .registryName(registryName)
-                .bucketId(bucketId)
-                .bucketName(bucketId)
-                .flowId(flowId)
-                .flowName(flowId)
-                .version(version)
-                .flowSnapshot(proposed)
-                .status(new StandardVersionedFlowStatus(flowState, flowState.getDescription()))
-                .build();
-
-            group.setVersionControlInformation(vci, Collections.emptyMap());
-        }
-
-        // Controller Services
-        // Controller Services have to be handled a bit differently than other components. This is because Processors and Controller
-        // Services may reference other Controller Services. Since we may be adding Service A, which depends on Service B, before adding
-        // Service B, we need to ensure that we create all Controller Services first and then call updateControllerService for each
-        // Controller Service. This way, we ensure that all services have been created before setting the properties. This allows us to
-        // properly obtain the correct mapping of Controller Service VersionedComponentID to Controller Service instance id.
-        final Map<String, ControllerServiceNode> servicesByVersionedId = group.getControllerServices(false).stream()
-            .collect(Collectors.toMap(component -> component.getVersionedComponentId().orElse(
-                NiFiRegistryFlowMapper.generateVersionedComponentId(component.getIdentifier())), Function.identity()));
-
-        final Set<String> controllerServicesRemoved = new HashSet<>(servicesByVersionedId.keySet());
-
-        final Map<ControllerServiceNode, VersionedControllerService> services = new HashMap<>();
-
-        // Add any Controller Service that does not yet exist.
-        final Map<String, ControllerServiceNode> servicesAdded = new HashMap<>();
-        for (final VersionedControllerService proposedService : proposed.getControllerServices()) {
-            ControllerServiceNode service = servicesByVersionedId.get(proposedService.getIdentifier());
-            if (service == null) {
-                service = addControllerService(group, proposedService, componentIdSeed);
-                LOG.info("Added {} to {}", service, group);
-                servicesAdded.put(proposedService.getIdentifier(), service);
-            }
-
-            services.put(service, proposedService);
-        }
-
-        // Because we don't know what order to instantiate the Controller Services, it's possible that we have two services such that Service A references Service B.
-        // If Service A happens to get created before Service B, the identifiers won't get matched up. As a result, we now iterate over all created Controller Services
-        // and update them again now that all Controller Services have been created at this level, so that the linkage can now be properly established.
-        for (final VersionedControllerService proposedService : proposed.getControllerServices()) {
-            final ControllerServiceNode addedService = servicesAdded.get(proposedService.getIdentifier());
-            if (addedService == null) {
-                continue;
-            }
-
-            updateControllerService(addedService, proposedService);
-        }
-
-        // Update all of the Controller Services to match the VersionedControllerService
-        for (final Map.Entry<ControllerServiceNode, VersionedControllerService> entry : services.entrySet()) {
-            final ControllerServiceNode service = entry.getKey();
-            final VersionedControllerService proposedService = entry.getValue();
-
-            if (updatedVersionedComponentIds.contains(proposedService.getIdentifier())) {
-                updateControllerService(service, proposedService);
-                LOG.info("Updated {}", service);
-            }
-
-            controllerServicesRemoved.remove(proposedService.getIdentifier());
-        }
-
-        // Before we can update child groups, we must first remove any connections that are connected to those child groups' input/output ports.
-        // We cannot add or update connections yet, though. That must be done at the end, as it's possible that the component that is the source/destination of the connection
-        // has not yet been added.
-        final Map<String, Connection> connectionsByVersionedId = group.getConnections().stream()
-            .collect(Collectors.toMap(component -> component.getVersionedComponentId().orElse(
-                NiFiRegistryFlowMapper.generateVersionedComponentId(component.getIdentifier())), Function.identity()));
-        final Set<String> connectionsRemoved = new HashSet<>(connectionsByVersionedId.keySet());
-
-        for (final VersionedConnection proposedConnection : proposed.getConnections()) {
-            connectionsRemoved.remove(proposedConnection.getIdentifier());
-        }
-
-        // Connections must be the first thing to remove, not the last. Otherwise, we will fail
-        // to remove a component if it has a connection going to it!
-        for (final String removedVersionedId : connectionsRemoved) {
-            final Connection connection = connectionsByVersionedId.get(removedVersionedId);
-            LOG.info("Removing {} from {}", connection, group);
-            group.removeConnection(connection);
-            flowManager.onConnectionRemoved(connection);
-        }
-
-        // Child groups
-        final Map<String, ProcessGroup> childGroupsByVersionedId = group.getProcessGroups().stream()
-            .collect(Collectors.toMap(component -> component.getVersionedComponentId().orElse(
-                NiFiRegistryFlowMapper.generateVersionedComponentId(component.getIdentifier())), Function.identity()));
-        final Set<String> childGroupsRemoved = new HashSet<>(childGroupsByVersionedId.keySet());
-
-        for (final VersionedProcessGroup proposedChildGroup : proposed.getProcessGroups()) {
-            final ProcessGroup childGroup = childGroupsByVersionedId.get(proposedChildGroup.getIdentifier());
-            final VersionedFlowCoordinates childCoordinates = proposedChildGroup.getVersionedFlowCoordinates();
-
-            // if there is a nested process group that is versioned controlled, make sure get the param contexts that go with that snapshot
-            // instead of the ones from the parent which would have been passed in to this method
-            Map<String, VersionedParameterContext> childParameterContexts = versionedParameterContexts;
-            if (childCoordinates != null && updateDescendantVersionedGroups) {
-                childParameterContexts = getVersionedParameterContexts(childCoordinates);
-            }
-
-            if (childGroup == null) {
-                final ProcessGroup added = addProcessGroup(group, proposedChildGroup, componentIdSeed, variablesToSkip, childParameterContexts);
-                flowManager.onProcessGroupAdded(added);
-                added.findAllRemoteProcessGroups().forEach(RemoteProcessGroup::initialize);
-                LOG.info("Added {} to {}", added, this);
-            } else if (childCoordinates == null || updateDescendantVersionedGroups) {
-                updateProcessGroup(childGroup, proposedChildGroup, componentIdSeed, updatedVersionedComponentIds, true, true, updateDescendantVersionedGroups,
-                    variablesToSkip, childParameterContexts);
-                LOG.info("Updated {}", childGroup);
-            }
-
-            childGroupsRemoved.remove(proposedChildGroup.getIdentifier());
-        }
-
-        // Funnels
-        final Map<String, Funnel> funnelsByVersionedId = group.getFunnels().stream()
-            .collect(Collectors.toMap(component -> component.getVersionedComponentId().orElse(
-                NiFiRegistryFlowMapper.generateVersionedComponentId(component.getIdentifier())), Function.identity()));
-        final Set<String> funnelsRemoved = new HashSet<>(funnelsByVersionedId.keySet());
-
-        for (final VersionedFunnel proposedFunnel : proposed.getFunnels()) {
-            final Funnel funnel = funnelsByVersionedId.get(proposedFunnel.getIdentifier());
-            if (funnel == null) {
-                final Funnel added = addFunnel(group, proposedFunnel, componentIdSeed);
-                flowManager.onFunnelAdded(added);
-                LOG.info("Added {} to {}", added, this);
-            } else if (updatedVersionedComponentIds.contains(proposedFunnel.getIdentifier())) {
-                updateFunnel(funnel, proposedFunnel);
-                LOG.info("Updated {}", funnel);
-            } else {
-                funnel.setPosition(new Position(proposedFunnel.getPosition().getX(), proposedFunnel.getPosition().getY()));
-            }
-
-            funnelsRemoved.remove(proposedFunnel.getIdentifier());
-        }
-
-        // Input Ports
-        final Map<String, Port> inputPortsByVersionedId = group.getInputPorts().stream()
-            .collect(Collectors.toMap(component -> component.getVersionedComponentId().orElse(
-                NiFiRegistryFlowMapper.generateVersionedComponentId(component.getIdentifier())), Function.identity()));
-        final Set<String> inputPortsRemoved = new HashSet<>(inputPortsByVersionedId.keySet());
-
-        for (final VersionedPort proposedPort : proposed.getInputPorts()) {
-            final Port port = inputPortsByVersionedId.get(proposedPort.getIdentifier());
-            if (port == null) {
-                final String temporaryName = generateTemporaryPortName(proposedPort);
-                final Port added = addInputPort(group, proposedPort, componentIdSeed, temporaryName);
-                proposedPortFinalNames.put(added, proposedPort.getName());
-                flowManager.onInputPortAdded(added);
-                LOG.info("Added {} to {}", added, this);
-            } else if (updatedVersionedComponentIds.contains(proposedPort.getIdentifier())) {
-                final String temporaryName = generateTemporaryPortName(proposedPort);
-                proposedPortFinalNames.put(port, proposedPort.getName());
-                updatePort(port, proposedPort, temporaryName);
-                LOG.info("Updated {}", port);
-            } else {
-                port.setPosition(new Position(proposedPort.getPosition().getX(), proposedPort.getPosition().getY()));
-            }
-
-            inputPortsRemoved.remove(proposedPort.getIdentifier());
-        }
-
-        // Output Ports
-        final Map<String, Port> outputPortsByVersionedId = group.getOutputPorts().stream()
-            .collect(Collectors.toMap(component -> component.getVersionedComponentId().orElse(
-                NiFiRegistryFlowMapper.generateVersionedComponentId(component.getIdentifier())), Function.identity()));
-        final Set<String> outputPortsRemoved = new HashSet<>(outputPortsByVersionedId.keySet());
-
-        for (final VersionedPort proposedPort : proposed.getOutputPorts()) {
-            final Port port = outputPortsByVersionedId.get(proposedPort.getIdentifier());
-            if (port == null) {
-                final String temporaryName = generateTemporaryPortName(proposedPort);
-                final Port added = addOutputPort(group, proposedPort, componentIdSeed, temporaryName);
-                proposedPortFinalNames.put(added, proposedPort.getName());
-                flowManager.onOutputPortAdded(added);
-                LOG.info("Added {} to {}", added, this);
-            } else if (updatedVersionedComponentIds.contains(proposedPort.getIdentifier())) {
-                final String temporaryName = generateTemporaryPortName(proposedPort);
-                proposedPortFinalNames.put(port, proposedPort.getName());
-                updatePort(port, proposedPort, temporaryName);
-                LOG.info("Updated {}", port);
-            } else {
-                port.setPosition(new Position(proposedPort.getPosition().getX(), proposedPort.getPosition().getY()));
-            }
-
-            outputPortsRemoved.remove(proposedPort.getIdentifier());
-        }
-
-        // Labels
-        final Map<String, Label> labelsByVersionedId = group.getLabels().stream()
-            .collect(Collectors.toMap(component -> component.getVersionedComponentId().orElse(
-                NiFiRegistryFlowMapper.generateVersionedComponentId(component.getIdentifier())), Function.identity()));
-        final Set<String> labelsRemoved = new HashSet<>(labelsByVersionedId.keySet());
-
-        for (final VersionedLabel proposedLabel : proposed.getLabels()) {
-            final Label label = labelsByVersionedId.get(proposedLabel.getIdentifier());
-            if (label == null) {
-                final Label added = addLabel(group, proposedLabel, componentIdSeed);
-                LOG.info("Added {} to {}", added, this);
-            } else if (updatedVersionedComponentIds.contains(proposedLabel.getIdentifier())) {
-                updateLabel(label, proposedLabel);
-                LOG.info("Updated {}", label);
-            } else {
-                label.setPosition(new Position(proposedLabel.getPosition().getX(), proposedLabel.getPosition().getY()));
-            }
-
-            labelsRemoved.remove(proposedLabel.getIdentifier());
-        }
-
-        // Processors
-        final Map<String, ProcessorNode> processorsByVersionedId = group.getProcessors().stream()
-            .collect(Collectors.toMap(component -> component.getVersionedComponentId().orElse(
-                NiFiRegistryFlowMapper.generateVersionedComponentId(component.getIdentifier())), Function.identity()));
-        final Set<String> processorsRemoved = new HashSet<>(processorsByVersionedId.keySet());
-        final Map<ProcessorNode, Set<Relationship>> autoTerminatedRelationships = new HashMap<>();
-
-        for (final VersionedProcessor proposedProcessor : proposed.getProcessors()) {
-            final ProcessorNode processor = processorsByVersionedId.get(proposedProcessor.getIdentifier());
-            if (processor == null) {
-                final ProcessorNode added = addProcessor(group, proposedProcessor, componentIdSeed);
-                flowManager.onProcessorAdded(added);
-
-                final Set<Relationship> proposedAutoTerminated =
-                    proposedProcessor.getAutoTerminatedRelationships() == null ? Collections.emptySet() : proposedProcessor.getAutoTerminatedRelationships().stream()
-                        .map(added::getRelationship)
-                        .collect(Collectors.toSet());
-                autoTerminatedRelationships.put(added, proposedAutoTerminated);
-                LOG.info("Added {} to {}", added, this);
-            } else if (updatedVersionedComponentIds.contains(proposedProcessor.getIdentifier())) {
-                updateProcessor(processor, proposedProcessor);
-
-                final Set<Relationship> proposedAutoTerminated =
-                    proposedProcessor.getAutoTerminatedRelationships() == null ? Collections.emptySet() : proposedProcessor.getAutoTerminatedRelationships().stream()
-                        .map(processor::getRelationship)
-                        .collect(Collectors.toSet());
-
-                if (!processor.getAutoTerminatedRelationships().equals(proposedAutoTerminated)) {
-                    autoTerminatedRelationships.put(processor, proposedAutoTerminated);
-                }
-
-                LOG.info("Updated {}", processor);
-            } else {
-                processor.setPosition(new Position(proposedProcessor.getPosition().getX(), proposedProcessor.getPosition().getY()));
-            }
-
-            processorsRemoved.remove(proposedProcessor.getIdentifier());
-        }
-
-        // Remote Groups
-        final Map<String, RemoteProcessGroup> rpgsByVersionedId = group.getRemoteProcessGroups().stream()
-            .collect(Collectors.toMap(component -> component.getVersionedComponentId().orElse(
-                NiFiRegistryFlowMapper.generateVersionedComponentId(component.getIdentifier())), Function.identity()));
-        final Set<String> rpgsRemoved = new HashSet<>(rpgsByVersionedId.keySet());
-
-        for (final VersionedRemoteProcessGroup proposedRpg : proposed.getRemoteProcessGroups()) {
-            final RemoteProcessGroup rpg = rpgsByVersionedId.get(proposedRpg.getIdentifier());
-            if (rpg == null) {
-                final RemoteProcessGroup added = addRemoteProcessGroup(group, proposedRpg, componentIdSeed);
-                LOG.info("Added {} to {}", added, this);
-            } else if (updatedVersionedComponentIds.contains(proposedRpg.getIdentifier())) {
-                updateRemoteProcessGroup(rpg, proposedRpg, componentIdSeed);
-                LOG.info("Updated {}", rpg);
-            } else {
-                rpg.setPosition(new Position(proposedRpg.getPosition().getX(), proposedRpg.getPosition().getY()));
-            }
-
-            rpgsRemoved.remove(proposedRpg.getIdentifier());
-        }
-
-        //Remove deletable Input and Output Ports.
-        //addConnection method may link the ports incorrectly, for example:
-        //Current flow: PGA IP1         ProcessGroupA has an Input Port IP1
-        //New flow: PGA P1-C1           ProcessGroupA has a new connection C1, its source is a new Processor P1
-        //            |     |           and its destination pointing to the moved Input Port IP1 under a new child ProcessGroup PGB
-        //          PGB    IP1
-        //As Input Port (IP1) originally belonged to PGA the new connection would be incorrectly linked to the old Input Port
-        //instead of the one being in PGB, so it needs to be removed first before updating the connections.
-
-        Iterator<String> inputPortsRemovedIterator = inputPortsRemoved.iterator();
-        while (inputPortsRemovedIterator.hasNext()) {
-            final String removedVersionedId = inputPortsRemovedIterator.next();
-            final Port port = inputPortsByVersionedId.get(removedVersionedId);
-            LOG.info("Removing {} from {}", port, group);
-            try {
-                group.removeInputPort(port);
-                inputPortsRemovedIterator.remove();
-            } catch (IllegalStateException e) {
-                LOG.info("Removing {} from {} not possible at the moment, will try again after updated the connections.", port, group);
-            }
-        }
-
-        Iterator<String> outputPortsRemovedIterator = outputPortsRemoved.iterator();
-        while (outputPortsRemovedIterator.hasNext()) {
-            final String removedVersionedId = outputPortsRemovedIterator.next();
-            final Port port = outputPortsByVersionedId.get(removedVersionedId);
-            LOG.info("Removing {} from {}", port, group);
-            try {
-                group.removeOutputPort(port);
-                outputPortsRemovedIterator.remove();
-            } catch (IllegalStateException e) {
-                LOG.info("Removing {} from {} not possible at the moment, will try again after updated the connections.", port, group);
-            }
-        }
-
-        // Add and update Connections
-        for (final VersionedConnection proposedConnection : proposed.getConnections()) {
-            final Connection connection = connectionsByVersionedId.get(proposedConnection.getIdentifier());
-            if (connection == null) {
-                final Connection added = addConnection(group, proposedConnection, componentIdSeed);
-                flowManager.onConnectionAdded(added);
-                LOG.info("Added {} to {}", added, this);
-            } else if (isUpdateable(connection)) {
-                // If the connection needs to be updated, then the source and destination will already have
-                // been stopped (else, the validation above would fail). So if the source or the destination is running,
-                // then we know that we don't need to update the connection.
-                updateConnection(connection, proposedConnection);
-                LOG.info("Updated {}", connection);
-            }
-        }
-
-        // Remove components that exist in the local flow but not the remote flow.
-
-        // Once the appropriate connections have been removed, we may now update Processors' auto-terminated relationships.
-        // We cannot do this above, in the 'updateProcessor' call because if a connection is removed and changed to auto-terminated,
-        // then updating this in the updateProcessor call above would attempt to set the Relationship to being auto-terminated while a
-        // Connection for that relationship exists. This will throw an Exception.
-        autoTerminatedRelationships.forEach(ProcessorNode::setAutoTerminatedRelationships);
-
-        // Remove all controller services no longer in use
-        for (final String removedVersionedId : controllerServicesRemoved) {
-            final ControllerServiceNode service = servicesByVersionedId.get(removedVersionedId);
-            LOG.info("Removing {} from {}", service, group);
-            // Must remove Controller Service through Flow Controller in order to remove from cache
-            controllerServiceProvider.removeControllerService(service);
-        }
-
-        for (final String removedVersionedId : funnelsRemoved) {
-            final Funnel funnel = funnelsByVersionedId.get(removedVersionedId);
-            LOG.info("Removing {} from {}", funnel, group);
-            group.removeFunnel(funnel);
-        }
-
-        //Removing remaining input ports
-        for (final String removedVersionedId : inputPortsRemoved) {
-            final Port port = inputPortsByVersionedId.get(removedVersionedId);
-            LOG.info("Removing {} from {}", port, group);
-            group.removeInputPort(port);
-        }
-
-        //Removing remaining output ports
-        for (final String removedVersionedId : outputPortsRemoved) {
-            final Port port = outputPortsByVersionedId.get(removedVersionedId);
-            LOG.info("Removing {} from {}", port, group);
-            group.removeOutputPort(port);
-        }
-
-        // Now that all input/output ports have been removed, we should be able to update
-        // all ports to the final name that was proposed in the new flow version.
-        for (final Map.Entry<Port, String> portAndFinalName : proposedPortFinalNames.entrySet()) {
-            final Port port = portAndFinalName.getKey();
-            final String finalName = portAndFinalName.getValue();
-            LOG.info("Updating {} to replace temporary name with final name", port);
-
-            // For public ports we need to consider if another public port exists somewhere else in the flow with the
-            // same name, and if so then rename the incoming port so the flow can still be imported
-            if (port instanceof PublicPort) {
-                final PublicPort publicPort = (PublicPort) port;
-                final String publicPortFinalName = getPublicPortFinalName(publicPort, finalName);
-                updatePortToSetFinalName(publicPort, publicPortFinalName);
-            } else {
-                updatePortToSetFinalName(port, finalName);
-            }
-        }
-
-        for (final String removedVersionedId : labelsRemoved) {
-            final Label label = labelsByVersionedId.get(removedVersionedId);
-            LOG.info("Removing {} from {}", label, group);
-            group.removeLabel(label);
-        }
-
-        for (final String removedVersionedId : processorsRemoved) {
-            final ProcessorNode processor = processorsByVersionedId.get(removedVersionedId);
-            LOG.info("Removing {} from {}", processor, group);
-            group.removeProcessor(processor);
-        }
-
-        for (final String removedVersionedId : rpgsRemoved) {
-            final RemoteProcessGroup rpg = rpgsByVersionedId.get(removedVersionedId);
-            LOG.info("Removing {} from {}", rpg, group);
-            group.removeRemoteProcessGroup(rpg);
-        }
-
-        for (final String removedVersionedId : childGroupsRemoved) {
-            final ProcessGroup childGroup = childGroupsByVersionedId.get(removedVersionedId);
-            LOG.info("Removing {} from {}", childGroup, group);
-            group.removeProcessGroup(childGroup);
-        }
-    }
-
-    private Map<String, VersionedParameterContext> getVersionedParameterContexts(final VersionedFlowCoordinates versionedFlowCoordinates) {
-        final String registryId = flowRegistryClient.getFlowRegistryId(versionedFlowCoordinates.getRegistryUrl());
-        if (registryId == null) {
-            throw new ResourceNotFoundException("Could not find any Flow Registry registered with url: " + versionedFlowCoordinates.getRegistryUrl());
-        }
-
-        final FlowRegistry flowRegistry = flowRegistryClient.getFlowRegistry(registryId);
-        if (flowRegistry == null) {
-            throw new ResourceNotFoundException("Could not find any Flow Registry registered with identifier " + registryId);
-        }
-
-        final String bucketId = versionedFlowCoordinates.getBucketId();
-        final String flowId = versionedFlowCoordinates.getFlowId();
-        final int flowVersion = versionedFlowCoordinates.getVersion();
-
-        try {
-            final VersionedFlowSnapshot childSnapshot = flowRegistry.getFlowContents(bucketId, flowId, flowVersion, false);
-            return childSnapshot.getParameterContexts();
-        } catch (final NiFiRegistryException e) {
-            throw new IllegalArgumentException("The Flow Registry with ID " + registryId + " reports that no Flow exists with Bucket "
-                + bucketId + ", Flow " + flowId + ", Version " + flowVersion, e);
-        } catch (final IOException ioe) {
-            throw new IllegalStateException(
-                "Failed to communicate with Flow Registry when attempting to retrieve a versioned flow");
-        }
-    }
-
-    private ParameterContext createParameterContext(final VersionedParameterContext versionedParameterContext, final String parameterContextId,
-                                                    final Map<String, VersionedParameterContext> versionedParameterContexts, final String componentIdSeed) {
-        final Map<String, Parameter> parameters = new HashMap<>();
-        for (final VersionedParameter versionedParameter : versionedParameterContext.getParameters()) {
-            final ParameterDescriptor descriptor = new ParameterDescriptor.Builder()
-                .name(versionedParameter.getName())
-                .description(versionedParameter.getDescription())
-                .sensitive(versionedParameter.isSensitive())
-                .build();
-
-            final Parameter parameter = new Parameter(descriptor, versionedParameter.getValue());
-            parameters.put(versionedParameter.getName(), parameter);
-        }
-        final List<ParameterContextReferenceEntity> parameterContextRefs = new ArrayList<>();
-        if (versionedParameterContext.getInheritedParameterContexts() != null) {
-            parameterContextRefs.addAll(versionedParameterContext.getInheritedParameterContexts().stream()
-                    .map(name -> createParameterReferenceEntity(name, versionedParameterContexts, componentIdSeed))
-                    .collect(Collectors.toList()));
-        }
-
-        return flowManager.createParameterContext(parameterContextId, versionedParameterContext.getName(), parameters, parameterContextRefs);
-    }
-
-    private ParameterContextReferenceEntity createParameterReferenceEntity(final String parameterContextName,
-                                                                           final Map<String, VersionedParameterContext> versionedParameterContexts,
-                                                                           final String componentIdSeed) {
-        final ParameterContextReferenceEntity entity = new ParameterContextReferenceEntity();
-        final ParameterContextReferenceDTO dto = new ParameterContextReferenceDTO();
-        final VersionedParameterContext versionedParameterContext = versionedParameterContexts.get(parameterContextName);
-        final ParameterContext selectedParameterContext = selectParameterContext(versionedParameterContext, componentIdSeed, versionedParameterContexts);
-        dto.setName(selectedParameterContext.getName());
-        dto.setId(selectedParameterContext.getIdentifier());
-        entity.setId(dto.getId());
-        entity.setComponent(dto);
-        return entity;
-    }
-
-    private void addMissingConfiguration(final VersionedParameterContext versionedParameterContext, final ParameterContext currentParameterContext,
-                                         final String componentIdSeed, final Map<String, VersionedParameterContext> versionedParameterContexts) {
-        final Map<String, Parameter> parameters = new HashMap<>();
-        for (final VersionedParameter versionedParameter : versionedParameterContext.getParameters()) {
-            final Optional<Parameter> parameterOption = currentParameterContext.getParameter(versionedParameter.getName());
-            if (parameterOption.isPresent()) {
-                // Skip this parameter, since it is already defined. We only want to add missing parameters
-                continue;
-            }
-
-            final ParameterDescriptor descriptor = new ParameterDescriptor.Builder()
-                .name(versionedParameter.getName())
-                .description(versionedParameter.getDescription())
-                .sensitive(versionedParameter.isSensitive())
-                .build();
-
-            final Parameter parameter = new Parameter(descriptor, versionedParameter.getValue());
-            parameters.put(versionedParameter.getName(), parameter);
-        }
-
-        currentParameterContext.setParameters(parameters);
-
-        // If the current parameter context doesn't have any inherited param contexts but the versioned one does,
-        // add the versioned ones.
-        if (versionedParameterContext.getInheritedParameterContexts() != null && !versionedParameterContext.getInheritedParameterContexts().isEmpty()
-                && currentParameterContext.getInheritedParameterContexts().isEmpty()) {
-            currentParameterContext.setInheritedParameterContexts(versionedParameterContext.getInheritedParameterContexts().stream()
-                    .map(name -> selectParameterContext(versionedParameterContexts.get(name), componentIdSeed, versionedParameterContexts))
-                    .collect(Collectors.toList()));
-        }
-    }
-
-    private ParameterContext getParameterContextByName(final String contextName) {
-        return flowManager.getParameterContextManager().getParameterContexts().stream()
-            .filter(context -> context.getName().equals(contextName))
-            .findAny()
-            .orElse(null);
-    }
-
-    private void updateParameterContext(final ProcessGroup group, final VersionedProcessGroup proposed, final Map<String, VersionedParameterContext> versionedParameterContexts,
-                                        final String componentIdSeed) {
-        // Update the Parameter Context
-        final ParameterContext currentParamContext = group.getParameterContext();
-        final String proposedParameterContextName = proposed.getParameterContextName();
-        if (proposedParameterContextName != null) {
-            if (currentParamContext == null) {
-                // Create a new Parameter Context based on the parameters provided
-                final VersionedParameterContext versionedParameterContext = versionedParameterContexts.get(proposedParameterContextName);
-
-                // Protect against NPE in the event somehow the proposed name is not in the set of contexts
-                if (versionedParameterContext == null) {
-                    final String paramContextNames = StringUtils.join(versionedParameterContexts.keySet());
-                    throw new IllegalStateException("Proposed parameter context name '" + proposedParameterContextName
-                        + "' does not exist in set of available parameter contexts [" + paramContextNames + "]");
-                }
-
-                final ParameterContext selectedParameterContext = selectParameterContext(versionedParameterContext, componentIdSeed, versionedParameterContexts);
-                group.setParameterContext(selectedParameterContext);
-            } else {
-                // Update the current Parameter Context so that it has any Parameters included in the proposed context
-                final VersionedParameterContext versionedParameterContext = versionedParameterContexts.get(proposedParameterContextName);
-                addMissingConfiguration(versionedParameterContext, currentParamContext, componentIdSeed, versionedParameterContexts);
-            }
-        }
-    }
-
-    private ParameterContext selectParameterContext(final VersionedParameterContext versionedParameterContext, final String componentIdSeed,
-                                                    final Map<String, VersionedParameterContext> versionedParameterContexts) {
-        final ParameterContext contextByName = getParameterContextByName(versionedParameterContext.getName());
-        final ParameterContext selectedParameterContext;
-        if (contextByName == null) {
-            final String parameterContextId = generateUuid(versionedParameterContext.getName(), versionedParameterContext.getName(), componentIdSeed);
-            selectedParameterContext = createParameterContext(versionedParameterContext, parameterContextId, versionedParameterContexts, componentIdSeed);
-        } else {
-            selectedParameterContext = contextByName;
-            addMissingConfiguration(versionedParameterContext, selectedParameterContext, componentIdSeed, versionedParameterContexts);
-        }
-        return selectedParameterContext;
-    }
-
-    private void updateVariableRegistry(final ProcessGroup group, final VersionedProcessGroup proposed, final Set<String> variablesToSkip) {
-        // Determine which variables have been added/removed and add/remove them from this group's variable registry.
-        // We don't worry about if a variable value has changed, because variables are designed to be 'environment specific.'
-        // As a result, once imported, we won't update variables to match the remote flow, but we will add any missing variables
-        // and remove any variables that are no longer part of the remote flow.
-        final Set<String> existingVariableNames = group.getVariableRegistry().getVariableMap().keySet().stream()
-            .map(VariableDescriptor::getName)
-            .collect(Collectors.toSet());
-
-        final Map<String, String> updatedVariableMap = new HashMap<>();
-
-        // If any new variables exist in the proposed flow, add those to the variable registry.
-        for (final Map.Entry<String, String> entry : proposed.getVariables().entrySet()) {
-            if (!existingVariableNames.contains(entry.getKey()) && !variablesToSkip.contains(entry.getKey())) {
-                updatedVariableMap.put(entry.getKey(), entry.getValue());
-            }
-        }
-
-        group.setVariables(updatedVariableMap);
-    }
-
-    private String getPublicPortFinalName(final PublicPort publicPort, final String proposedFinalName) {
-        final Optional<Port> existingPublicPort;
-        if (TransferDirection.RECEIVE == publicPort.getDirection()) {
-            existingPublicPort = flowManager.getPublicInputPort(proposedFinalName);
-        } else {
-            existingPublicPort = flowManager.getPublicOutputPort(proposedFinalName);
-        }
-
-        if (existingPublicPort.isPresent() && !existingPublicPort.get().getIdentifier().equals(publicPort.getIdentifier())) {
-            return getPublicPortFinalName(publicPort, "Copy of " + proposedFinalName);
-        } else {
-            return proposedFinalName;
-        }
-    }
-
-    private boolean isUpdateable(final Connection connection) {
-        final Connectable source = connection.getSource();
-        if (source.getConnectableType() != ConnectableType.FUNNEL && source.isRunning()) {
-            return false;
-        }
-
-        final Connectable destination = connection.getDestination();
-        return destination.getConnectableType() == ConnectableType.FUNNEL || !destination.isRunning();
-    }
-
-    private String generateTemporaryPortName(final VersionedPort proposedPort) {
-        final String versionedPortId = proposedPort.getIdentifier();
-        final String proposedPortFinalName = proposedPort.getName();
-        return proposedPortFinalName + " (" + versionedPortId + ")";
-    }
-
-    private void updatePortToSetFinalName(final Port port, final String name) {
-        writeLock.lock();
-        try {
-            port.setName(name);
-        } finally {
-            writeLock.unlock();
-        }
-    }
-
     private String generateUuid(final String propposedId, final String destinationGroupId, final String seed) {
         long msb = UUID.nameUUIDFromBytes((propposedId + destinationGroupId).getBytes(StandardCharsets.UTF_8)).getMostSignificantBits();
 
@@ -4629,573 +3882,6 @@ public final class StandardProcessGroup implements ProcessGroup {
         return uuid.toString();
     }
 
-    private ProcessGroup addProcessGroup(final ProcessGroup destination, final VersionedProcessGroup proposed, final String componentIdSeed, final Set<String> variablesToSkip,
-                                         final Map<String, VersionedParameterContext> versionedParameterContexts)
-        throws ProcessorInstantiationException {
-        final ProcessGroup group = flowManager.createProcessGroup(generateUuid(proposed.getIdentifier(), destination.getIdentifier(), componentIdSeed));
-        group.setVersionedComponentId(proposed.getIdentifier());
-        group.setParent(destination);
-        updateProcessGroup(group, proposed, componentIdSeed, Collections.emptySet(), true, true, true, variablesToSkip, versionedParameterContexts);
-        destination.addProcessGroup(group);
-        return group;
-    }
-
-    private void updateConnection(final Connection connection, final VersionedConnection proposed) {
-        connection.setBendPoints(proposed.getBends() == null ? Collections.emptyList() :
-            proposed.getBends().stream()
-                .map(pos -> new Position(pos.getX(), pos.getY()))
-                .collect(Collectors.toList()));
-
-        connection.setDestination(getConnectable(connection.getProcessGroup(), proposed.getDestination()));
-        connection.setLabelIndex(proposed.getLabelIndex());
-        connection.setName(proposed.getName());
-        connection.setRelationships(proposed.getSelectedRelationships().stream()
-            .map(name -> new Relationship.Builder().name(name).build())
-            .collect(Collectors.toSet()));
-        connection.setZIndex(proposed.getzIndex());
-
-        final FlowFileQueue queue = connection.getFlowFileQueue();
-        queue.setBackPressureDataSizeThreshold(proposed.getBackPressureDataSizeThreshold());
-        queue.setBackPressureObjectThreshold(proposed.getBackPressureObjectThreshold());
-        queue.setFlowFileExpiration(proposed.getFlowFileExpiration());
-
-        final List<FlowFilePrioritizer> prioritizers = proposed.getPrioritizers() == null ? Collections.emptyList() : proposed.getPrioritizers().stream()
-            .map(prioritizerName -> {
-                try {
-                    return flowManager.createPrioritizer(prioritizerName);
-                } catch (final Exception e) {
-                    throw new IllegalStateException("Failed to create Prioritizer of type " + prioritizerName + " for Connection with ID " + connection.getIdentifier());
-                }
-            })
-            .collect(Collectors.toList());
-
-        queue.setPriorities(prioritizers);
-
-        final String loadBalanceStrategyName = proposed.getLoadBalanceStrategy();
-        if (loadBalanceStrategyName == null) {
-            queue.setLoadBalanceStrategy(LoadBalanceStrategy.DO_NOT_LOAD_BALANCE, proposed.getPartitioningAttribute());
-        } else {
-            final LoadBalanceStrategy loadBalanceStrategy = LoadBalanceStrategy.valueOf(loadBalanceStrategyName);
-            final String partitioningAttribute = proposed.getPartitioningAttribute();
-
-            queue.setLoadBalanceStrategy(loadBalanceStrategy, partitioningAttribute);
-        }
-
-        final String compressionName = proposed.getLoadBalanceCompression();
-        if (compressionName == null) {
-            queue.setLoadBalanceCompression(LoadBalanceCompression.DO_NOT_COMPRESS);
-        } else {
-            queue.setLoadBalanceCompression(LoadBalanceCompression.valueOf(compressionName));
-        }
-    }
-
-    private Connection addConnection(final ProcessGroup destinationGroup, final VersionedConnection proposed, final String componentIdSeed) {
-        final Connectable source = getConnectable(destinationGroup, proposed.getSource());
-        if (source == null) {
-            throw new IllegalArgumentException("Connection has a source with identifier " + proposed.getIdentifier()
-                + " but no component could be found in the Process Group with a corresponding identifier");
-        }
-
-        final Connectable destination = getConnectable(destinationGroup, proposed.getDestination());
-        if (destination == null) {
-            throw new IllegalArgumentException("Connection has a destination with identifier " + proposed.getDestination().getId()
-                + " but no component could be found in the Process Group with a corresponding identifier");
-        }
-
-        final Connection connection = flowManager.createConnection(generateUuid(proposed.getIdentifier(), destination.getIdentifier(), componentIdSeed), proposed.getName(), source, destination,
-            proposed.getSelectedRelationships());
-        connection.setVersionedComponentId(proposed.getIdentifier());
-        destinationGroup.addConnection(connection);
-        updateConnection(connection, proposed);
-
-        flowManager.onConnectionAdded(connection);
-        return connection;
-    }
-
-    private Connectable getConnectable(final ProcessGroup group, final ConnectableComponent connectableComponent) {
-        final String id = connectableComponent.getId();
-
-        switch (connectableComponent.getType()) {
-            case FUNNEL:
-                return group.getFunnels().stream()
-                    .filter(component -> id.equals(component.getVersionedComponentId().orElse(
-                        NiFiRegistryFlowMapper.generateVersionedComponentId(component.getIdentifier()))))
-                    .findAny()
-                    .orElse(null);
-            case INPUT_PORT: {
-                final Optional<Port> port = group.getInputPorts().stream()
-                    .filter(component -> id.equals(component.getVersionedComponentId().orElse(
-                        NiFiRegistryFlowMapper.generateVersionedComponentId(component.getIdentifier()))))
-                    .findAny();
-
-                if (port.isPresent()) {
-                    return port.get();
-                }
-
-                // Attempt to locate child group by versioned component id
-                final Optional<ProcessGroup> optionalSpecifiedGroup = group.getProcessGroups().stream()
-                    .filter(child -> child.getVersionedComponentId().orElse(
-                        NiFiRegistryFlowMapper.generateVersionedComponentId(child.getIdentifier())).equals(connectableComponent.getGroupId()))
-                    .findFirst();
-
-                if (optionalSpecifiedGroup.isPresent()) {
-                    final ProcessGroup specifiedGroup = optionalSpecifiedGroup.get();
-                    return specifiedGroup.getInputPorts().stream()
-                        .filter(component -> id.equals(component.getVersionedComponentId().orElse(
-                            NiFiRegistryFlowMapper.generateVersionedComponentId(component.getIdentifier()))))
-                        .findAny()
-                        .orElse(null);
-                }
-
-                // If no child group matched the versioned component id, then look at all child groups. This is done because
-                // in older versions, we did not properly map Versioned Component ID's to Ports' parent groups. As a result,
-                // if the flow doesn't contain the properly mapped group id, we need to search all child groups.
-                return group.getProcessGroups().stream()
-                    .flatMap(gr -> gr.getInputPorts().stream())
-                    .filter(component -> id.equals(component.getVersionedComponentId().orElse(
-                        NiFiRegistryFlowMapper.generateVersionedComponentId(component.getIdentifier()))))
-                    .findAny()
-                    .orElse(null);
-            }
-            case OUTPUT_PORT: {
-                final Optional<Port> port = group.getOutputPorts().stream()
-                    .filter(component -> id.equals(component.getVersionedComponentId().orElse(
-                        NiFiRegistryFlowMapper.generateVersionedComponentId(component.getIdentifier()))))
-                    .findAny();
-
-                if (port.isPresent()) {
-                    return port.get();
-                }
-
-                // Attempt to locate child group by versioned component id
-                final Optional<ProcessGroup> optionalSpecifiedGroup = group.getProcessGroups().stream()
-                    .filter(child -> child.getVersionedComponentId().orElse(
-                        NiFiRegistryFlowMapper.generateVersionedComponentId(child.getIdentifier())).equals(connectableComponent.getGroupId()))
-                    .findFirst();
-
-                if (optionalSpecifiedGroup.isPresent()) {
-                    final ProcessGroup specifiedGroup = optionalSpecifiedGroup.get();
-                    return specifiedGroup.getOutputPorts().stream()
-                        .filter(component -> id.equals(component.getVersionedComponentId().orElse(
-                            NiFiRegistryFlowMapper.generateVersionedComponentId(component.getIdentifier()))))
-                        .findAny()
-                        .orElse(null);
-                }
-
-                // If no child group matched the versioned component id, then look at all child groups. This is done because
-                // in older versions, we did not properly map Versioned Component ID's to Ports' parent groups. As a result,
-                // if the flow doesn't contain the properly mapped group id, we need to search all child groups.
-                return group.getProcessGroups().stream()
-                    .flatMap(gr -> gr.getOutputPorts().stream())
-                    .filter(component -> id.equals(component.getVersionedComponentId().orElse(
-                        NiFiRegistryFlowMapper.generateVersionedComponentId(component.getIdentifier()))))
-                    .findAny()
-                    .orElse(null);
-            }
-            case PROCESSOR:
-                return group.getProcessors().stream()
-                    .filter(component -> id.equals(component.getVersionedComponentId().orElse(
-                        NiFiRegistryFlowMapper.generateVersionedComponentId(component.getIdentifier()))))
-                    .findAny()
-                    .orElse(null);
-            case REMOTE_INPUT_PORT: {
-                final String rpgId = connectableComponent.getGroupId();
-                final Optional<RemoteProcessGroup> rpgOption = group.getRemoteProcessGroups().stream()
-                    .filter(component -> rpgId.equals(component.getVersionedComponentId().orElse(
-                        NiFiRegistryFlowMapper.generateVersionedComponentId(component.getIdentifier()))))
-                    .findAny();
-
-                if (!rpgOption.isPresent()) {
-                    throw new IllegalArgumentException("Connection refers to a Port with ID " + id + " within Remote Process Group with ID "
-                        + rpgId + " but could not find a Remote Process Group corresponding to that ID");
-                }
-
-                final RemoteProcessGroup rpg = rpgOption.get();
-                final Optional<RemoteGroupPort> portByIdOption = rpg.getInputPorts().stream()
-                    .filter(component -> id.equals(component.getVersionedComponentId().orElse(
-                        NiFiRegistryFlowMapper.generateVersionedComponentId(component.getIdentifier()))))
-                    .findAny();
-
-                if (portByIdOption.isPresent()) {
-                    return portByIdOption.get();
-                }
-
-                return rpg.getInputPorts().stream()
-                    .filter(component -> connectableComponent.getName().equals(component.getName()))
-                    .findAny()
-                    .orElse(null);
-            }
-            case REMOTE_OUTPUT_PORT: {
-                final String rpgId = connectableComponent.getGroupId();
-                final Optional<RemoteProcessGroup> rpgOption = group.getRemoteProcessGroups().stream()
-                    .filter(component -> rpgId.equals(component.getVersionedComponentId().orElse(
-                        NiFiRegistryFlowMapper.generateVersionedComponentId(component.getIdentifier()))))
-                    .findAny();
-
-                if (!rpgOption.isPresent()) {
-                    throw new IllegalArgumentException("Connection refers to a Port with ID " + id + " within Remote Process Group with ID "
-                        + rpgId + " but could not find a Remote Process Group corresponding to that ID");
-                }
-
-                final RemoteProcessGroup rpg = rpgOption.get();
-                final Optional<RemoteGroupPort> portByIdOption = rpg.getOutputPorts().stream()
-                    .filter(component -> id.equals(component.getVersionedComponentId().orElse(
-                        NiFiRegistryFlowMapper.generateVersionedComponentId(component.getIdentifier()))))
-                    .findAny();
-
-                if (portByIdOption.isPresent()) {
-                    return portByIdOption.get();
-                }
-
-                return rpg.getOutputPorts().stream()
-                    .filter(component -> connectableComponent.getName().equals(component.getName()))
-                    .findAny()
-                    .orElse(null);
-            }
-        }
-
-        return null;
-    }
-
-    private void updateControllerService(final ControllerServiceNode service, final VersionedControllerService proposed) {
-        service.pauseValidationTrigger();
-        try {
-            service.setAnnotationData(proposed.getAnnotationData());
-            service.setComments(proposed.getComments());
-            service.setName(proposed.getName());
-
-            final Map<String, String> properties = populatePropertiesMap(service, proposed.getProperties(), proposed.getPropertyDescriptors(), service.getProcessGroup());
-            service.setProperties(properties, true);
-
-            if (!isEqual(service.getBundleCoordinate(), proposed.getBundle())) {
-                final BundleCoordinate newBundleCoordinate = toCoordinate(proposed.getBundle());
-                final List<PropertyDescriptor> descriptors = new ArrayList<>(service.getRawPropertyValues().keySet());
-                final Set<URL> additionalUrls = service.getAdditionalClasspathResources(descriptors);
-                reloadComponent.reload(service, proposed.getType(), newBundleCoordinate, additionalUrls);
-            }
-        } finally {
-            service.resumeValidationTrigger();
-        }
-    }
-
-    private boolean isEqual(final BundleCoordinate coordinate, final Bundle bundle) {
-        if (!bundle.getGroup().equals(coordinate.getGroup())) {
-            return false;
-        }
-
-        if (!bundle.getArtifact().equals(coordinate.getId())) {
-            return false;
-        }
-
-        return bundle.getVersion().equals(coordinate.getVersion());
-    }
-
-    private BundleCoordinate toCoordinate(final Bundle bundle) {
-        return new BundleCoordinate(bundle.getGroup(), bundle.getArtifact(), bundle.getVersion());
-    }
-
-    private ControllerServiceNode addControllerService(final ProcessGroup destination, final VersionedControllerService proposed, final String componentIdSeed) {
-        final String type = proposed.getType();
-        final String id = generateUuid(proposed.getIdentifier(), destination.getIdentifier(), componentIdSeed);
-
-        final Bundle bundle = proposed.getBundle();
-        final BundleCoordinate coordinate = toCoordinate(bundle);
-        final boolean firstTimeAdded = true;
-        final Set<URL> additionalUrls = Collections.emptySet();
-
-        final ControllerServiceNode newService = flowManager.createControllerService(type, id, coordinate, additionalUrls, firstTimeAdded, true, null);
-        newService.setVersionedComponentId(proposed.getIdentifier());
-
-        destination.addControllerService(newService);
-        updateControllerService(newService, proposed);
-
-        return newService;
-    }
-
-    private void updateFunnel(final Funnel funnel, final VersionedFunnel proposed) {
-        funnel.setPosition(new Position(proposed.getPosition().getX(), proposed.getPosition().getY()));
-    }
-
-    private Funnel addFunnel(final ProcessGroup destination, final VersionedFunnel proposed, final String componentIdSeed) {
-        final Funnel funnel = flowManager.createFunnel(generateUuid(proposed.getIdentifier(), destination.getIdentifier(), componentIdSeed));
-        funnel.setVersionedComponentId(proposed.getIdentifier());
-        destination.addFunnel(funnel);
-        updateFunnel(funnel, proposed);
-
-        return funnel;
-    }
-
-    private void updatePort(final Port port, final VersionedPort proposed, final String temporaryName) {
-        final String name = temporaryName != null ? temporaryName : proposed.getName();
-        port.setComments(proposed.getComments());
-        port.setName(name);
-        port.setPosition(new Position(proposed.getPosition().getX(), proposed.getPosition().getY()));
-        port.setMaxConcurrentTasks(proposed.getConcurrentlySchedulableTaskCount());
-        if (org.apache.nifi.flow.ScheduledState.DISABLED == proposed.getScheduledState()) {
-            port.disable();
-        }
-    }
-
-    private Port addInputPort(final ProcessGroup destination, final VersionedPort proposed, final String componentIdSeed, final String temporaryName) {
-        final String name = temporaryName != null ? temporaryName : proposed.getName();
-
-        final Port port;
-        if (proposed.isAllowRemoteAccess()) {
-            port = flowManager.createPublicInputPort(generateUuid(proposed.getIdentifier(), destination.getIdentifier(), componentIdSeed), name);
-        } else {
-            port = flowManager.createLocalInputPort(generateUuid(proposed.getIdentifier(), destination.getIdentifier(), componentIdSeed), name);
-        }
-
-        port.setVersionedComponentId(proposed.getIdentifier());
-        destination.addInputPort(port);
-        updatePort(port, proposed, temporaryName);
-
-        return port;
-    }
-
-    private Port addOutputPort(final ProcessGroup destination, final VersionedPort proposed, final String componentIdSeed, final String temporaryName) {
-        final String name = temporaryName != null ? temporaryName : proposed.getName();
-
-        final Port port;
-        if (proposed.isAllowRemoteAccess()) {
-            port = flowManager.createPublicOutputPort(generateUuid(proposed.getIdentifier(), destination.getIdentifier(), componentIdSeed), name);
-        } else {
-            port = flowManager.createLocalOutputPort(generateUuid(proposed.getIdentifier(), destination.getIdentifier(), componentIdSeed), name);
-        }
-
-        port.setVersionedComponentId(proposed.getIdentifier());
-        destination.addOutputPort(port);
-        updatePort(port, proposed, temporaryName);
-
-        return port;
-    }
-
-    private Label addLabel(final ProcessGroup destination, final VersionedLabel proposed, final String componentIdSeed) {
-        final Label label = flowManager.createLabel(generateUuid(proposed.getIdentifier(), destination.getIdentifier(), componentIdSeed), proposed.getLabel());
-        label.setVersionedComponentId(proposed.getIdentifier());
-        destination.addLabel(label);
-        updateLabel(label, proposed);
-
-        return label;
-    }
-
-    private void updateLabel(final Label label, final VersionedLabel proposed) {
-        label.setPosition(new Position(proposed.getPosition().getX(), proposed.getPosition().getY()));
-        label.setSize(new Size(proposed.getWidth(), proposed.getHeight()));
-        label.setStyle(proposed.getStyle());
-        label.setValue(proposed.getLabel());
-    }
-
-    private ProcessorNode addProcessor(final ProcessGroup destination, final VersionedProcessor proposed, final String componentIdSeed) throws ProcessorInstantiationException {
-        final BundleCoordinate coordinate = toCoordinate(proposed.getBundle());
-        final ProcessorNode procNode = flowManager.createProcessor(proposed.getType(), generateUuid(proposed.getIdentifier(), destination.getIdentifier(), componentIdSeed), coordinate, true);
-        procNode.setVersionedComponentId(proposed.getIdentifier());
-
-        destination.addProcessor(procNode);
-        updateProcessor(procNode, proposed);
-        // Notify the processor node that the configuration (properties, e.g.) has been restored
-        final StandardProcessContext processContext = new StandardProcessContext(procNode, controllerServiceProvider, encryptor,
-                stateManagerProvider.getStateManager(procNode.getProcessor().getIdentifier()), () -> false, nodeTypeProvider);
-        procNode.onConfigurationRestored(processContext);
-
-        return procNode;
-    }
-
-    private void updateProcessor(final ProcessorNode processor, final VersionedProcessor proposed) throws ProcessorInstantiationException {
-        processor.pauseValidationTrigger();
-        try {
-            processor.setAnnotationData(proposed.getAnnotationData());
-            processor.setBulletinLevel(LogLevel.valueOf(proposed.getBulletinLevel()));
-            processor.setComments(proposed.getComments());
-            processor.setName(proposed.getName());
-            processor.setPenalizationPeriod(proposed.getPenaltyDuration());
-
-            final Map<String, String> properties = populatePropertiesMap(processor, proposed.getProperties(), proposed.getPropertyDescriptors(), processor.getProcessGroup());
-            processor.setProperties(properties, true);
-            processor.setRunDuration(proposed.getRunDurationMillis(), TimeUnit.MILLISECONDS);
-            processor.setSchedulingStrategy(SchedulingStrategy.valueOf(proposed.getSchedulingStrategy()));
-            processor.setScheduldingPeriod(proposed.getSchedulingPeriod());
-            processor.setMaxConcurrentTasks(proposed.getConcurrentlySchedulableTaskCount());
-            processor.setExecutionNode(ExecutionNode.valueOf(proposed.getExecutionNode()));
-            processor.setStyle(proposed.getStyle());
-            processor.setYieldPeriod(proposed.getYieldDuration());
-            processor.setPosition(new Position(proposed.getPosition().getX(), proposed.getPosition().getY()));
-
-            if (proposed.getScheduledState() == org.apache.nifi.flow.ScheduledState.DISABLED) {
-                processor.getProcessGroup().disableProcessor(processor);
-            } else if (processor.getScheduledState() == ScheduledState.DISABLED) {
-                processor.getProcessGroup().enableProcessor(processor);
-            }
-
-            if (!isEqual(processor.getBundleCoordinate(), proposed.getBundle())) {
-                final BundleCoordinate newBundleCoordinate = toCoordinate(proposed.getBundle());
-                final List<PropertyDescriptor> descriptors = new ArrayList<>(processor.getProperties().keySet());
-                final Set<URL> additionalUrls = processor.getAdditionalClasspathResources(descriptors);
-                reloadComponent.reload(processor, proposed.getType(), newBundleCoordinate, additionalUrls);
-            }
-        } finally {
-            processor.resumeValidationTrigger();
-        }
-    }
-
-    private Map<String, String> populatePropertiesMap(final ComponentNode componentNode, final Map<String, String> proposedProperties,
-                                                      final Map<String, VersionedPropertyDescriptor> proposedDescriptors, final ProcessGroup group) {
-
-        // since VersionedPropertyDescriptor currently doesn't know if it is sensitive or not,
-        // keep track of which property descriptors are sensitive from the current properties
-        final Set<String> sensitiveProperties = new HashSet<>();
-
-        final Map<String, String> fullPropertyMap = new HashMap<>();
-        for (final PropertyDescriptor property : componentNode.getRawPropertyValues().keySet()) {
-            if (property.isSensitive()) {
-                sensitiveProperties.add(property.getName());
-            } else {
-                fullPropertyMap.put(property.getName(), null);
-            }
-        }
-
-        if (proposedProperties != null) {
-            // Build a Set of all properties that are included in either the currently configured property values or the proposed values.
-            final Set<String> updatedPropertyNames = new HashSet<>();
-            updatedPropertyNames.addAll(proposedProperties.keySet());
-            componentNode.getProperties().keySet().stream()
-                .map(PropertyDescriptor::getName)
-                .forEach(updatedPropertyNames::add);
-
-            for (final String propertyName : updatedPropertyNames) {
-                final VersionedPropertyDescriptor descriptor = proposedDescriptors.get(propertyName);
-
-                String value;
-                if (descriptor != null && descriptor.getIdentifiesControllerService()) {
-
-                    // Need to determine if the component's property descriptor for this service is already set to an id
-                    // of an existing service that is outside the current processor group, and if it is we want to leave
-                    // the property set to that value
-                    String existingExternalServiceId = null;
-                    final PropertyDescriptor componentDescriptor = componentNode.getPropertyDescriptor(propertyName);
-                    if (componentDescriptor != null) {
-                        final String componentDescriptorValue = componentNode.getEffectivePropertyValue(componentDescriptor);
-                        if (componentDescriptorValue != null) {
-                            final ControllerServiceNode serviceNode = findAncestorControllerService(componentDescriptorValue, getParent());
-                            if (serviceNode != null) {
-                                existingExternalServiceId = componentDescriptorValue;
-                            }
-                        }
-                    }
-
-                    // If the component's property descriptor is not already set to an id of an existing external service,
-                    // then we need to take the Versioned Component ID and resolve this to the instance ID of the service
-                    if (existingExternalServiceId == null) {
-                        final String serviceVersionedComponentId = proposedProperties.get(propertyName);
-                        String instanceId = getServiceInstanceId(serviceVersionedComponentId, group);
-                        value = instanceId == null ? serviceVersionedComponentId : instanceId;
-                    } else {
-                        value = existingExternalServiceId;
-                    }
-
-                } else {
-                    value = proposedProperties.get(propertyName);
-                }
-
-                // skip any sensitive properties that are not populated so we can retain whatever is currently set. We do this because sensitive properties are not stored in the registry
-                // unless the value is a reference to a Parameter. If the value in the registry is null, it indicates that the sensitive value was removed, so we want to keep the currently
-                // populated value. The exception to this rule is if the currently configured value is a Parameter Reference and the Versioned Flow is empty. In this case, it implies
-                // that the Versioned Flow has changed from a Parameter Reference to an explicit value. In this case, we do in fact want to change the value of the Sensitive Property from
-                // the current parameter reference to an unset value.
-                if (sensitiveProperties.contains(propertyName) && value == null) {
-                    final PropertyConfiguration propertyConfiguration = componentNode.getProperty(componentNode.getPropertyDescriptor(propertyName));
-                    if (propertyConfiguration == null) {
-                        continue;
-                    }
-
-                    // No parameter references. Property currently is set to an explicit value. We don't want to change it.
-                    if (propertyConfiguration.getParameterReferences().isEmpty()) {
-                        continue;
-                    }
-
-                    // Once we reach this point, the property is configured to reference a Parameter, and the value in the Versioned Flow is an explicit value,
-                    // so we want to continue on and update the value to null.
-                }
-
-                fullPropertyMap.put(propertyName, value);
-            }
-        }
-
-        return fullPropertyMap;
-    }
-
-    private String getServiceInstanceId(final String serviceVersionedComponentId, final ProcessGroup group) {
-        for (final ControllerServiceNode serviceNode : group.getControllerServices(false)) {
-            final String versionedId = serviceNode.getVersionedComponentId().orElse(
-                NiFiRegistryFlowMapper.generateVersionedComponentId(serviceNode.getIdentifier()));
-            if (versionedId.equals(serviceVersionedComponentId)) {
-                return serviceNode.getIdentifier();
-            }
-        }
-
-        final ProcessGroup parent = group.getParent();
-        if (parent == null) {
-            return null;
-        }
-
-        return getServiceInstanceId(serviceVersionedComponentId, parent);
-
-    }
-
-    private RemoteProcessGroup addRemoteProcessGroup(final ProcessGroup destination, final VersionedRemoteProcessGroup proposed, final String componentIdSeed) {
-        final RemoteProcessGroup rpg = flowManager.createRemoteProcessGroup(generateUuid(proposed.getIdentifier(), destination.getIdentifier(), componentIdSeed), proposed.getTargetUris());
-        rpg.setVersionedComponentId(proposed.getIdentifier());
-
-        destination.addRemoteProcessGroup(rpg);
-        updateRemoteProcessGroup(rpg, proposed, componentIdSeed);
-
-        return rpg;
-    }
-
-    private void updateRemoteProcessGroup(final RemoteProcessGroup rpg, final VersionedRemoteProcessGroup proposed, final String componentIdSeed) {
-        rpg.setComments(proposed.getComments());
-        rpg.setCommunicationsTimeout(proposed.getCommunicationsTimeout());
-        rpg.setInputPorts(proposed.getInputPorts() == null ? Collections.emptySet() : proposed.getInputPorts().stream()
-            .map(port -> createPortDescriptor(port, componentIdSeed, rpg.getIdentifier()))
-            .collect(Collectors.toSet()), false);
-        rpg.setName(proposed.getName());
-        rpg.setNetworkInterface(proposed.getLocalNetworkInterface());
-        rpg.setOutputPorts(proposed.getOutputPorts() == null ? Collections.emptySet() : proposed.getOutputPorts().stream()
-            .map(port -> createPortDescriptor(port, componentIdSeed, rpg.getIdentifier()))
-            .collect(Collectors.toSet()), false);
-        rpg.setPosition(new Position(proposed.getPosition().getX(), proposed.getPosition().getY()));
-        rpg.setProxyHost(proposed.getProxyHost());
-        rpg.setProxyPort(proposed.getProxyPort());
-        rpg.setProxyUser(proposed.getProxyUser());
-        rpg.setTransportProtocol(SiteToSiteTransportProtocol.valueOf(proposed.getTransportProtocol()));
-        rpg.setYieldDuration(proposed.getYieldDuration());
-    }
-
-    private RemoteProcessGroupPortDescriptor createPortDescriptor(final VersionedRemoteGroupPort proposed, final String componentIdSeed, final String rpgId) {
-        final StandardRemoteProcessGroupPortDescriptor descriptor = new StandardRemoteProcessGroupPortDescriptor();
-        descriptor.setVersionedComponentId(proposed.getIdentifier());
-
-        final BatchSize batchSize = proposed.getBatchSize();
-        if (batchSize != null) {
-            descriptor.setBatchCount(batchSize.getCount());
-            descriptor.setBatchDuration(batchSize.getDuration());
-            descriptor.setBatchSize(batchSize.getSize());
-        }
-
-        descriptor.setComments(proposed.getComments());
-        descriptor.setConcurrentlySchedulableTaskCount(proposed.getConcurrentlySchedulableTaskCount());
-        descriptor.setGroupId(proposed.getRemoteGroupId());
-        descriptor.setTargetId(proposed.getTargetId());
-        descriptor.setId(generateUuid(proposed.getIdentifier(), rpgId, componentIdSeed));
-        descriptor.setName(proposed.getName());
-        descriptor.setUseCompression(proposed.isUseCompression());
-
-        final boolean transmitting = org.apache.nifi.flow.ScheduledState.ENABLED == proposed.getScheduledState();
-        descriptor.setTransmitting(transmitting);
-
-        return descriptor;
-    }
-
     private Set<FlowDifference> getModifications() {
         final StandardVersionControlInformation vci = versionControlInfo.get();
 
@@ -5224,16 +3910,10 @@ public final class StandardProcessGroup implements ProcessGroup {
             final ComparableDataFlow currentFlow = new StandardComparableDataFlow("Local Flow", versionedGroup);
             final ComparableDataFlow snapshotFlow = new StandardComparableDataFlow("Versioned Flow", vci.getFlowSnapshot());
 
-            final FlowComparator flowComparator = new StandardFlowComparator(snapshotFlow, currentFlow, getAncestorServiceIds(), new EvolvingDifferenceDescriptor());
+            final FlowComparator flowComparator = new StandardFlowComparator(snapshotFlow, currentFlow, getAncestorServiceIds(), new EvolvingDifferenceDescriptor(), encryptor::decrypt);
             final FlowComparison comparison = flowComparator.compare();
             final Set<FlowDifference> differences = comparison.getDifferences().stream()
-                .filter(difference -> difference.getDifferenceType() != DifferenceType.BUNDLE_CHANGED)
-                .filter(FlowDifferenceFilters.FILTER_ADDED_REMOVED_REMOTE_PORTS)
-                .filter(FlowDifferenceFilters.FILTER_PUBLIC_PORT_NAME_CHANGES)
-                .filter(FlowDifferenceFilters.FILTER_IGNORABLE_VERSIONED_FLOW_COORDINATE_CHANGES)
-                .filter(diff -> !FlowDifferenceFilters.isNewPropertyWithDefaultValue(diff, flowManager))
-                .filter(diff -> !FlowDifferenceFilters.isNewRelationshipAutoTerminatedAndDefaulted(diff, versionedGroup, flowManager))
-                .filter(diff -> !FlowDifferenceFilters.isScheduledStateNew(diff))
+                .filter(difference -> !FlowDifferenceFilters.isEnvironmentalChange(difference, versionedGroup, flowManager))
                 .collect(Collectors.toCollection(HashSet::new));
 
             LOG.debug("There are {} differences between this Local Flow and the Versioned Flow: {}", differences.size(), differences);
@@ -5279,232 +3959,30 @@ public final class StandardProcessGroup implements ProcessGroup {
                 verifyNoDescendantsWithLocalModifications("be updated");
             }
 
-            final VersionedProcessGroup flowContents = updatedFlow.getFlowContents();
-
-            // Ensure no deleted child process groups contain templates and optionally no deleted connections contain data
-            // in their queue. Note that this check enforces ancestry among the group components to avoid a scenario where
-            // a component is matched by id, but it does not exist in the same hierarchy and thus will be removed and
-            // re-added when the update is performed
-            verifyCanRemoveMissingComponents(this, flowContents, verifyConnectionRemoval);
-
-            // Determine which input ports were removed from this process group
-            final Map<String, Port> removedInputPortsByVersionId = new HashMap<>();
-            getInputPorts()
-                .forEach(port -> removedInputPortsByVersionId.put(port.getVersionedComponentId().orElse(
-                    NiFiRegistryFlowMapper.generateVersionedComponentId(port.getIdentifier())), port));
-            flowContents.getInputPorts().stream()
-                .map(VersionedPort::getIdentifier)
-                .forEach(removedInputPortsByVersionId::remove);
-
-            // Ensure that there are no incoming connections for any Input Port that was removed.
-            for (final Port inputPort : removedInputPortsByVersionId.values()) {
-                final List<Connection> incomingConnections = inputPort.getIncomingConnections();
-                if (!incomingConnections.isEmpty()) {
-                    throw new IllegalStateException(this + " cannot be updated to the proposed flow because the proposed flow "
-                        + "does not contain the Input Port " + inputPort + " and the Input Port currently has an incoming connection");
-                }
-            }
-
-            // Determine which output ports were removed from this process group
-            final Map<String, Port> removedOutputPortsByVersionId = new HashMap<>();
-            getOutputPorts()
-                .forEach(port -> removedOutputPortsByVersionId.put(port.getVersionedComponentId().orElse(
-                    NiFiRegistryFlowMapper.generateVersionedComponentId(port.getIdentifier())), port));
-            flowContents.getOutputPorts().stream()
-                .map(VersionedPort::getIdentifier)
-                .forEach(removedOutputPortsByVersionId::remove);
-
-            // Ensure that there are no outgoing connections for any Output Port that was removed.
-            for (final Port outputPort : removedOutputPortsByVersionId.values()) {
-                final Set<Connection> outgoingConnections = outputPort.getConnections();
-                if (!outgoingConnections.isEmpty()) {
-                    throw new IllegalStateException(this + " cannot be updated to the proposed flow because the proposed flow "
-                        + "does not contain the Output Port " + outputPort + " and the Output Port currently has an outgoing connection");
-                }
-            }
-
-            // Ensure that all Processors are instantiable
-            final Map<String, VersionedProcessor> proposedProcessors = new HashMap<>();
-            findAllProcessors(flowContents, proposedProcessors);
-
-            findAllProcessors()
-                .forEach(proc -> proposedProcessors.remove(proc.getVersionedComponentId().orElse(
-                    NiFiRegistryFlowMapper.generateVersionedComponentId(proc.getIdentifier()))));
-
-            for (final VersionedProcessor processorToAdd : proposedProcessors.values()) {
-                final String processorToAddClass = processorToAdd.getType();
-                final BundleCoordinate processorToAddCoordinate = toCoordinate(processorToAdd.getBundle());
-
-                // Get the exact bundle requested, if it exists.
-                final Bundle bundle = processorToAdd.getBundle();
-                final BundleCoordinate coordinate = new BundleCoordinate(bundle.getGroup(), bundle.getArtifact(), bundle.getVersion());
-                final org.apache.nifi.bundle.Bundle resolved = extensionManager.getBundle(coordinate);
+            final ComponentIdGenerator componentIdGenerator = (proposedId, instanceId, destinationGroupId) -> proposedId;
+            final ProcessGroupSynchronizationContext groupSynchronizationContext = createGroupSynchronizationContext(
+                componentIdGenerator, ComponentScheduler.NOP_SCHEDULER, FlowMappingOptions.DEFAULT_OPTIONS);
+            final StandardProcessGroupSynchronizer synchronizer = new StandardProcessGroupSynchronizer(groupSynchronizationContext);
 
-                if (resolved == null) {
-                    // Could not resolve the bundle explicitly. Check for possible bundles.
-                    final List<org.apache.nifi.bundle.Bundle> possibleBundles = extensionManager.getBundles(processorToAddClass);
-                    final boolean bundleExists = possibleBundles.stream()
-                        .anyMatch(b -> processorToAddCoordinate.equals(b.getBundleDetails().getCoordinate()));
-
-                    if (!bundleExists && possibleBundles.size() != 1) {
-                        throw new IllegalArgumentException("Unknown bundle " + processorToAddCoordinate.toString() + " for processor type " + processorToAddClass);
-                    }
-                }
-            }
-
-            // Ensure that all Controller Services are instantiable
-            final Map<String, VersionedControllerService> proposedServices = new HashMap<>();
-            findAllControllerServices(flowContents, proposedServices);
-
-            findAllControllerServices()
-                .forEach(service -> proposedServices.remove(service.getVersionedComponentId().orElse(
-                    NiFiRegistryFlowMapper.generateVersionedComponentId(service.getIdentifier()))));
-
-            for (final VersionedControllerService serviceToAdd : proposedServices.values()) {
-                final String serviceToAddClass = serviceToAdd.getType();
-                final BundleCoordinate serviceToAddCoordinate = toCoordinate(serviceToAdd.getBundle());
-
-                final org.apache.nifi.bundle.Bundle resolved = extensionManager.getBundle(serviceToAddCoordinate);
-                if (resolved == null) {
-                    final List<org.apache.nifi.bundle.Bundle> possibleBundles = extensionManager.getBundles(serviceToAddClass);
-                    final boolean bundleExists = possibleBundles.stream()
-                        .anyMatch(b -> serviceToAddCoordinate.equals(b.getBundleDetails().getCoordinate()));
-
-                    if (!bundleExists && possibleBundles.size() != 1) {
-                        throw new IllegalArgumentException("Unknown bundle " + serviceToAddCoordinate.toString() + " for service type " + serviceToAddClass);
-                    }
-                }
-            }
-
-            // Ensure that all Prioritizers are instantiate-able and that any load balancing configuration is correct
-            // Enforcing ancestry on connection matching here is not important because all we're interested in is locating
-            // new prioritizers and load balance strategy types so if a matching connection existed anywhere in the current
-            // flow, then its prioritizer and load balance strategy are already validated
-            final Map<String, VersionedConnection> proposedConnections = new HashMap<>();
-            findAllConnections(flowContents, proposedConnections);
-
-            findAllConnections()
-                .forEach(conn -> proposedConnections.remove(conn.getVersionedComponentId().orElse(
-                    NiFiRegistryFlowMapper.generateVersionedComponentId(conn.getIdentifier()))));
-
-            for (final VersionedConnection connectionToAdd : proposedConnections.values()) {
-                if (connectionToAdd.getPrioritizers() != null) {
-                    for (final String prioritizerType : connectionToAdd.getPrioritizers()) {
-                        try {
-                            flowManager.createPrioritizer(prioritizerType);
-                        } catch (Exception e) {
-                            throw new IllegalArgumentException("Unable to create Prioritizer of type " + prioritizerType, e);
-                        }
-                    }
-                }
-
-                final String loadBalanceStrategyName = connectionToAdd.getLoadBalanceStrategy();
-                if (loadBalanceStrategyName != null) {
-                    try {
-                        LoadBalanceStrategy.valueOf(loadBalanceStrategyName);
-                    } catch (final IllegalArgumentException iae) {
-                        throw new IllegalArgumentException("Unable to create Connection with Load Balance Strategy of '" + loadBalanceStrategyName
-                            + "' because this is not a known Load Balance Strategy");
-                    }
-                }
-            }
+            synchronizer.verifyCanSynchronize(this, updatedFlow.getFlowContents(), verifyConnectionRemoval);
         } finally {
             readLock.unlock();
         }
     }
 
-    private void findAllProcessors(final VersionedProcessGroup group, final Map<String, VersionedProcessor> map) {
-        for (final VersionedProcessor processor : group.getProcessors()) {
-            map.put(processor.getIdentifier(), processor);
-        }
-
-        for (final VersionedProcessGroup childGroup : group.getProcessGroups()) {
-            findAllProcessors(childGroup, map);
-        }
-    }
-
-    private void findAllControllerServices(final VersionedProcessGroup group, final Map<String, VersionedControllerService> map) {
-        for (final VersionedControllerService service : group.getControllerServices()) {
-            map.put(service.getIdentifier(), service);
-        }
-
-        for (final VersionedProcessGroup childGroup : group.getProcessGroups()) {
-            findAllControllerServices(childGroup, map);
-        }
-    }
-
-    private void findAllConnections(final VersionedProcessGroup group, final Map<String, VersionedConnection> map) {
-        for (final VersionedConnection connection : group.getConnections()) {
-            map.put(connection.getIdentifier(), connection);
-        }
-
-        for (final VersionedProcessGroup childGroup : group.getProcessGroups()) {
-            findAllConnections(childGroup, map);
-        }
-    }
-
-    /**
-     * Match components of the given process group to the proposed versioned process group and verify missing components
-     * are in a state that they can be safely removed. Specifically, check for removed child process groups and descendants.
-     * Disallow removal of groups with attached templates. Optionally also check for removed connections with data in their
-     * queue, either because the connections were removed from a matched process group or their group itself was removed.
-     *
-     * @param processGroup the current process group to examine
-     * @param proposedGroup the proposed versioned process group to match with
-     * @param verifyConnectionRemoval whether or not to verify that connections that are not present in the proposed flow can be removed
-     */
-    private void verifyCanRemoveMissingComponents(final ProcessGroup processGroup, final VersionedProcessGroup proposedGroup,
-                                                  final boolean verifyConnectionRemoval) {
-        if (verifyConnectionRemoval) {
-            final Map<String, VersionedConnection> proposedConnectionsByVersionedId = proposedGroup.getConnections().stream()
-                .collect(Collectors.toMap(component -> component.getIdentifier(), Function.identity()));
-
-            // match group's current connections to proposed connections to determine if they've been removed
-            for (final Connection connection : processGroup.getConnections()) {
-                final String versionedId = connection.getVersionedComponentId().orElse(
-                    NiFiRegistryFlowMapper.generateVersionedComponentId(connection.getIdentifier()));
-                final VersionedConnection proposedConnection = proposedConnectionsByVersionedId.get(versionedId);
-                if (proposedConnection == null) {
-                    // connection doesn't exist in proposed connections, make sure it doesn't have any data in it
-                    final FlowFileQueue flowFileQueue = connection.getFlowFileQueue();
-                    if (!flowFileQueue.isEmpty()) {
-                        throw new IllegalStateException(this + " cannot be updated to the proposed flow because the proposed flow "
-                            + "does not contain a match for " + connection + " and the connection currently has data in the queue.");
-                    }
-                }
-            }
-        }
-
-        final Map<String, VersionedProcessGroup> proposedGroupsByVersionedId = proposedGroup.getProcessGroups().stream()
-            .collect(Collectors.toMap(component -> component.getIdentifier(), Function.identity()));
-
-        // match current child groups to proposed child groups to determine if they've been removed
-        for (final ProcessGroup childGroup : processGroup.getProcessGroups()) {
-            final String versionedId = childGroup.getVersionedComponentId().orElse(
-                NiFiRegistryFlowMapper.generateVersionedComponentId(childGroup.getIdentifier()));
-            final VersionedProcessGroup proposedChildGroup = proposedGroupsByVersionedId.get(versionedId);
-            if (proposedChildGroup == null) {
-                // child group will be removed, check group and descendants for attached templates
-                final Template removedTemplate = findAllTemplates(childGroup).stream().findFirst().orElse(null);
-                if (removedTemplate != null) {
-                    throw new IllegalStateException(this + " cannot be updated to the proposed flow because the child " + removedTemplate.getProcessGroup()
-                        + " that exists locally has one or more Templates, and the proposed flow does not contain these templates. "
-                        + "A Process Group cannot be deleted while it contains Templates. Please remove the Templates before re-attempting.");
-                }
-                if (verifyConnectionRemoval) {
-                    // check removed group and its descendants for connections with data in the queue
-                    final Connection removedConnection = findAllConnections(childGroup).stream()
-                        .filter(connection -> !connection.getFlowFileQueue().isEmpty()).findFirst().orElse(null);
-                    if (removedConnection != null) {
-                        throw new IllegalStateException(this + " cannot be updated to the proposed flow because the proposed flow "
-                            + "does not contain a match for " + removedConnection + " and the connection currently has data in the queue.");
-                    }
-                }
-            } else {
-                // child group successfully matched, recurse into verification of its contents
-                verifyCanRemoveMissingComponents(childGroup, proposedChildGroup, verifyConnectionRemoval);
-            }
-        }
+    private ProcessGroupSynchronizationContext createGroupSynchronizationContext(final ComponentIdGenerator componentIdGenerator, final ComponentScheduler componentScheduler,
+                                                                                 final FlowMappingOptions flowMappingOptions) {
+        return new ProcessGroupSynchronizationContext.Builder()
+            .componentIdGenerator(componentIdGenerator)
+            .flowManager(flowManager)
+            .flowRegistryClient(flowRegistryClient)
+            .reloadComponent(reloadComponent)
+            .controllerServiceProvider(controllerServiceProvider)
+            .extensionManager(extensionManager)
+            .componentScheduler(componentScheduler)
+            .flowMappingOptions(flowMappingOptions)
+            .processContextFactory(this::createProcessContext)
+            .build();
     }
 
     @Override
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/groups/StandardProcessGroupSynchronizer.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/groups/StandardProcessGroupSynchronizer.java
new file mode 100644
index 0000000..e768fda
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/groups/StandardProcessGroupSynchronizer.java
@@ -0,0 +1,2084 @@
+/*
+ * 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.groups;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.bundle.BundleCoordinate;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.connectable.Connectable;
+import org.apache.nifi.connectable.ConnectableType;
+import org.apache.nifi.connectable.Connection;
+import org.apache.nifi.connectable.Funnel;
+import org.apache.nifi.connectable.Port;
+import org.apache.nifi.connectable.Position;
+import org.apache.nifi.connectable.Size;
+import org.apache.nifi.controller.ComponentNode;
+import org.apache.nifi.controller.ProcessorNode;
+import org.apache.nifi.controller.PropertyConfiguration;
+import org.apache.nifi.controller.ScheduledState;
+import org.apache.nifi.controller.Template;
+import org.apache.nifi.controller.exception.ProcessorInstantiationException;
+import org.apache.nifi.controller.label.Label;
+import org.apache.nifi.controller.queue.FlowFileQueue;
+import org.apache.nifi.controller.queue.LoadBalanceCompression;
+import org.apache.nifi.controller.queue.LoadBalanceStrategy;
+import org.apache.nifi.controller.service.ControllerServiceNode;
+import org.apache.nifi.controller.service.ControllerServiceState;
+import org.apache.nifi.encrypt.EncryptionException;
+import org.apache.nifi.flow.BatchSize;
+import org.apache.nifi.flow.Bundle;
+import org.apache.nifi.flow.ComponentType;
+import org.apache.nifi.flow.ConnectableComponent;
+import org.apache.nifi.flow.VersionedComponent;
+import org.apache.nifi.flow.VersionedConnection;
+import org.apache.nifi.flow.VersionedControllerService;
+import org.apache.nifi.flow.VersionedFlowCoordinates;
+import org.apache.nifi.flow.VersionedFunnel;
+import org.apache.nifi.flow.VersionedLabel;
+import org.apache.nifi.flow.VersionedPort;
+import org.apache.nifi.flow.VersionedProcessGroup;
+import org.apache.nifi.flow.VersionedProcessor;
+import org.apache.nifi.flow.VersionedPropertyDescriptor;
+import org.apache.nifi.flow.VersionedRemoteGroupPort;
+import org.apache.nifi.flow.VersionedRemoteProcessGroup;
+import org.apache.nifi.flowfile.FlowFilePrioritizer;
+import org.apache.nifi.logging.LogLevel;
+import org.apache.nifi.parameter.Parameter;
+import org.apache.nifi.parameter.ParameterContext;
+import org.apache.nifi.parameter.ParameterDescriptor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.registry.VariableDescriptor;
+import org.apache.nifi.registry.client.NiFiRegistryException;
+import org.apache.nifi.registry.flow.FlowRegistry;
+import org.apache.nifi.registry.flow.StandardVersionControlInformation;
+import org.apache.nifi.registry.flow.VersionControlInformation;
+import org.apache.nifi.registry.flow.VersionedFlowSnapshot;
+import org.apache.nifi.registry.flow.VersionedFlowState;
+import org.apache.nifi.registry.flow.VersionedParameter;
+import org.apache.nifi.registry.flow.VersionedParameterContext;
+import org.apache.nifi.registry.flow.diff.ComparableDataFlow;
+import org.apache.nifi.registry.flow.diff.DifferenceType;
+import org.apache.nifi.registry.flow.diff.FlowComparator;
+import org.apache.nifi.registry.flow.diff.FlowComparison;
+import org.apache.nifi.registry.flow.diff.FlowDifference;
+import org.apache.nifi.registry.flow.diff.StandardComparableDataFlow;
+import org.apache.nifi.registry.flow.diff.StandardFlowComparator;
+import org.apache.nifi.registry.flow.diff.StaticDifferenceDescriptor;
+import org.apache.nifi.registry.flow.mapping.NiFiRegistryFlowMapper;
+import org.apache.nifi.remote.PublicPort;
+import org.apache.nifi.remote.RemoteGroupPort;
+import org.apache.nifi.remote.StandardRemoteProcessGroupPortDescriptor;
+import org.apache.nifi.remote.TransferDirection;
+import org.apache.nifi.remote.protocol.SiteToSiteTransportProtocol;
+import org.apache.nifi.scheduling.ExecutionNode;
+import org.apache.nifi.scheduling.SchedulingStrategy;
+import org.apache.nifi.util.FlowDifferenceFilters;
+import org.apache.nifi.web.ResourceNotFoundException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+public class StandardProcessGroupSynchronizer implements ProcessGroupSynchronizer {
+    private static final Logger LOG = LoggerFactory.getLogger(StandardProcessGroupSynchronizer.class);
+    private static final String TEMP_FUNNEL_ID_SUFFIX = "-temp-funnel";
+    public static final String ENC_PREFIX = "enc{";
+    public static final String ENC_SUFFIX = "}";
+
+    private final ProcessGroupSynchronizationContext context;
+    private final Set<String> updatedVersionedComponentIds = new HashSet<>();
+
+    private Set<String> preExistingVariables = new HashSet<>();
+    private GroupSynchronizationOptions syncOptions;
+
+    public StandardProcessGroupSynchronizer(final ProcessGroupSynchronizationContext context) {
+        this.context = context;
+    }
+
+    private void setPreExistingVariables(final Set<String> preExistingVariables) {
+        this.preExistingVariables = preExistingVariables;
+    }
+
+    private void setUpdatedVersionedComponentIds(final Set<String> updatedVersionedComponentIds) {
+        this.updatedVersionedComponentIds.clear();
+        this.updatedVersionedComponentIds.addAll(updatedVersionedComponentIds);
+    }
+
+    public void setSynchronizationOptions(final GroupSynchronizationOptions syncOptions) {
+        this.syncOptions = syncOptions;
+    }
+
+    @Override
+    public void synchronize(final ProcessGroup group, final VersionedFlowSnapshot proposedSnapshot, final GroupSynchronizationOptions options) {
+
+        final NiFiRegistryFlowMapper mapper = new NiFiRegistryFlowMapper(context.getExtensionManager(), context.getFlowMappingOptions());
+        final VersionedProcessGroup versionedGroup = mapper.mapProcessGroup(group, context.getControllerServiceProvider(), context.getFlowRegistryClient(), true);
+
+        final ComparableDataFlow localFlow = new StandardComparableDataFlow("Currently Loaded Flow", versionedGroup);
+        final ComparableDataFlow proposedFlow = new StandardComparableDataFlow("Proposed Flow", proposedSnapshot.getFlowContents());
+
+        final PropertyDecryptor decryptor = options.getPropertyDecryptor();
+        final FlowComparator flowComparator = new StandardFlowComparator(proposedFlow, localFlow, group.getAncestorServiceIds(), new StaticDifferenceDescriptor(), decryptor::decrypt);
+        final FlowComparison flowComparison = flowComparator.compare();
+
+        updatedVersionedComponentIds.clear();
+        setSynchronizationOptions(options);
+
+        for (final FlowDifference diff : flowComparison.getDifferences()) {
+            if (FlowDifferenceFilters.isPropertyMissingFromGhostComponent(diff, context.getFlowManager())) {
+                continue;
+            }
+            if (FlowDifferenceFilters.isScheduledStateNew(diff)) {
+                continue;
+            }
+
+            // If this update adds a new Controller Service, then we need to check if the service already exists at a higher level
+            // and if so compare our VersionedControllerService to the existing service.
+            if (diff.getDifferenceType() == DifferenceType.COMPONENT_ADDED) {
+                final VersionedComponent component = diff.getComponentA() == null ? diff.getComponentB() : diff.getComponentA();
+                if (ComponentType.CONTROLLER_SERVICE == component.getComponentType()) {
+                    final ControllerServiceNode serviceNode = getVersionedControllerService(group, component.getIdentifier());
+                    if (serviceNode != null) {
+                        final VersionedControllerService versionedService = mapper.mapControllerService(serviceNode, context.getControllerServiceProvider(),
+                            Collections.singleton(serviceNode.getProcessGroupIdentifier()), new HashMap<>());
+                        final Set<FlowDifference> differences = flowComparator.compareControllerServices(versionedService, (VersionedControllerService) component);
+
+                        if (!differences.isEmpty()) {
+                            updatedVersionedComponentIds.add(component.getIdentifier());
+                        }
+
+                        continue;
+                    }
+                }
+            }
+
+            final VersionedComponent component = diff.getComponentA() == null ? diff.getComponentB() : diff.getComponentA();
+            updatedVersionedComponentIds.add(component.getIdentifier());
+
+            if (component.getComponentType() == ComponentType.REMOTE_INPUT_PORT || component.getComponentType() == ComponentType.REMOTE_OUTPUT_PORT) {
+                final String remoteGroupId = ((VersionedRemoteGroupPort) component).getRemoteGroupId();
+                updatedVersionedComponentIds.add(remoteGroupId);
+            }
+        }
+
+        if (LOG.isInfoEnabled()) {
+            final String differencesByLine = flowComparison.getDifferences().stream()
+                .map(FlowDifference::toString)
+                .collect(Collectors.joining("\n"));
+
+            LOG.info("Updating {} to {}; there are {} differences to take into account:\n{}", group, proposedSnapshot,
+                flowComparison.getDifferences().size(), differencesByLine);
+        }
+
+        final Set<String> knownVariables = getKnownVariableNames(group);
+
+        preExistingVariables.clear();
+
+        // If we don't want to update existing variables, we need to populate the pre-existing variables so that we know which variables already existed.
+        // We can't do this when updating the Variable Registry for a Process Group because variables are inherited, and the variables of the parent group
+        // may already have been updated when we get to the point of updating a child's Variable Registry. As a result, we build up a Set of all known
+        // Variables before we update the Variable Registries.
+        if (!options.isUpdateExistingVariables()) {
+            preExistingVariables.addAll(knownVariables);
+        }
+
+        context.getFlowManager().withParameterContextResolution(() -> {
+            try {
+                synchronize(group, proposedSnapshot.getFlowContents(), proposedSnapshot.getParameterContexts());
+            } catch (final ProcessorInstantiationException pie) {
+                throw new RuntimeException(pie);
+            }
+        });
+
+        group.onComponentModified();
+    }
+
+    private void synchronize(final ProcessGroup group, final VersionedProcessGroup proposed, final Map<String, VersionedParameterContext> versionedParameterContexts)
+                    throws ProcessorInstantiationException {
+
+        // Some components, such as Processors, may have a Scheduled State of RUNNING in the proposed flow. However, if we
+        // transition the service into the RUNNING state, and then we need to update a Connection that is connected to it,
+        // updating the Connection will fail because the Connection's source & destination must both be stopped in order to
+        // update it. To avoid that, we simply pause the scheduler. Once all updates have been made, we will resume the scheduler.
+        context.getComponentScheduler().pause();
+
+        group.setComments(proposed.getComments());
+
+        if (syncOptions.isUpdateSettings()) {
+            if (proposed.getName() != null) {
+                group.setName(proposed.getName());
+            }
+
+            if (proposed.getPosition() != null) {
+                group.setPosition(new Position(proposed.getPosition().getX(), proposed.getPosition().getY()));
+            }
+        }
+
+        // Ensure that we create all Parameter Contexts before updating them. This is necessary in case the proposed incoming dataflow has
+        // parameter contexts that inherit from one another and neither the inheriting nor inherited parameter context exists.
+        versionedParameterContexts.values().forEach(this::createParameterContextWithoutReferences);
+
+        updateParameterContext(group, proposed, versionedParameterContexts, context.getComponentIdGenerator());
+        updateVariableRegistry(group, proposed);
+
+        final FlowFileConcurrency flowFileConcurrency = proposed.getFlowFileConcurrency() == null ? FlowFileConcurrency.UNBOUNDED :
+            FlowFileConcurrency.valueOf(proposed.getFlowFileConcurrency());
+        group.setFlowFileConcurrency(flowFileConcurrency);
+
+        final FlowFileOutboundPolicy outboundPolicy = proposed.getFlowFileOutboundPolicy() == null ? FlowFileOutboundPolicy.STREAM_WHEN_AVAILABLE :
+            FlowFileOutboundPolicy.valueOf(proposed.getFlowFileOutboundPolicy());
+        group.setFlowFileOutboundPolicy(outboundPolicy);
+
+        group.setDefaultFlowFileExpiration(proposed.getDefaultFlowFileExpiration());
+        group.setDefaultBackPressureObjectThreshold(proposed.getDefaultBackPressureObjectThreshold());
+        group.setDefaultBackPressureDataSizeThreshold(proposed.getDefaultBackPressureDataSizeThreshold());
+
+        final VersionedFlowCoordinates remoteCoordinates = proposed.getVersionedFlowCoordinates();
+        if (remoteCoordinates == null) {
+            group.disconnectVersionControl(false);
+        } else {
+            final String registryId = context.getFlowRegistryClient().getFlowRegistryId(remoteCoordinates.getRegistryUrl());
+            final String bucketId = remoteCoordinates.getBucketId();
+            final String flowId = remoteCoordinates.getFlowId();
+            final int version = remoteCoordinates.getVersion();
+
+            final FlowRegistry flowRegistry = context.getFlowRegistryClient().getFlowRegistry(registryId);
+            final String registryName = flowRegistry == null ? registryId : flowRegistry.getName();
+
+            final VersionedFlowState flowState;
+            if (remoteCoordinates.getLatest() == null) {
+                flowState = VersionedFlowState.SYNC_FAILURE;
+            } else {
+                flowState = remoteCoordinates.getLatest() ? VersionedFlowState.UP_TO_DATE : VersionedFlowState.STALE;
+            }
+
+            final VersionControlInformation vci = new StandardVersionControlInformation.Builder()
+                .registryId(registryId)
+                .registryName(registryName)
+                .bucketId(bucketId)
+                .bucketName(bucketId)
+                .flowId(flowId)
+                .flowName(flowId)
+                .version(version)
+                .flowSnapshot(syncOptions.isUpdateGroupVersionControlSnapshot() ? proposed : null)
+                .status(new StandardVersionedFlowStatus(flowState, flowState.getDescription()))
+                .build();
+
+            group.setVersionControlInformation(vci, Collections.emptyMap());
+        }
+
+        // In order to properly update all of the components, we have to follow a specific order of operations, in order to ensure that
+        // we don't try to perform illegal operations like removing a Processor that has an incoming connection (which would throw an
+        // IllegalStateException and fail).
+        //
+        // The sequence of steps / order of operations are as follows:
+        //
+        // 1. Remove any Controller Services that do not exist in the proposed group
+        // 2. Add any Controller Services that are in the proposed group that are not in the current flow
+        // 3. Update Controller Services to match those in the proposed group
+        // 4. Remove any connections that do not exist in the proposed group
+        // 5. For any connection that does exist, if the proposed group has a different destination for the connection, update the destination.
+        //    If the new destination does not yet exist in the flow, set the destination as some temporary component.
+        // 6. Remove any other components that do not exist in the proposed group.
+        // 7. Add any components, other than Connections, that exist in the proposed group but not in the current flow
+        // 8. Update components, other than Connections, to match those in the proposed group
+        // 9. Add connections that exist in the proposed group that are not in the current flow
+        // 10. Update connections to match those in the proposed group
+        // 11. Delete the temporary destination that was created above
+
+        // Keep track of any processors that have been updated to have auto-terminated relationships so that we can set those
+        // auto-terminated relationships after we've handled creating/deleting necessary connections.
+        final Map<ProcessorNode, Set<Relationship>> autoTerminatedRelationships = new HashMap<>();
+
+        // During the flow update, we will use temporary names for process group ports. This is because port names must be
+        // unique within a process group, but during an update we might temporarily be in a state where two ports have the same name.
+        // For example, if a process group update involves removing/renaming port A, and then adding/updating port B where B is given
+        // A's former name. This is a valid state by the end of the flow update, but for a brief moment there may be two ports with the
+        // same name. To avoid this conflict, we keep the final names in a map indexed by port id, use a temporary name for each port
+        // during the update, and after all ports have been added/updated/removed, we set the final names on all ports.
+        final Map<Port, String> proposedPortFinalNames = new HashMap<>();
+
+        // Controller Services
+        final Map<String, ControllerServiceNode> controllerServicesByVersionedId = componentsById(group, grp -> grp.getControllerServices(false),
+            ControllerServiceNode::getIdentifier, ControllerServiceNode::getVersionedComponentId);
+        removeMissingControllerServices(group, proposed, controllerServicesByVersionedId);
+        synchronizeControllerServices(group, proposed, controllerServicesByVersionedId);
+
+        // Remove any connections that are not in the Proposed Process Group
+        // Connections must be the first thing to remove, not the last. Otherwise, we will fail
+        // to remove a component if it has a connection going to it!
+        final Map<String, Connection> connectionsByVersionedId = componentsById(group, ProcessGroup::getConnections, Connection::getIdentifier, Connection::getVersionedComponentId);
+        removeMissingConnections(group, proposed, connectionsByVersionedId);
+
+        // Before we remove other components, we have to ensure that the Connections have the appropriate destinations. Otherwise, we could have a situation
+        // where Connection A used to have a destination of B but now has a destination of C, which doesn't exist yet. And B doesn't exist in the new flow.
+        // This is a problem because we cannot remove B, since it has an incoming Connection. And we can't change the destination to C because C hasn't been
+        // added yet. As a result, we need a temporary location to set as the Connection's destination. So we create a Funnel for this and then we can update
+        // all Connections to have the appropriate destinations.
+        final Set<String> connectionsWithTempDestination = updateConnectionDestinations(group, proposed, connectionsByVersionedId);
+
+        try {
+            final Map<String, Funnel> funnelsByVersionedId = componentsById(group, ProcessGroup::getFunnels);
+            final Map<String, ProcessorNode> processorsByVersionedId = componentsById(group, ProcessGroup::getProcessors);
+            final Map<String, Port> inputPortsByVersionedId = componentsById(group, ProcessGroup::getInputPorts);
+            final Map<String, Port> outputPortsByVersionedId = componentsById(group, ProcessGroup::getOutputPorts);
+            final Map<String, Label> labelsByVersionedId = componentsById(group, ProcessGroup::getLabels, Label::getIdentifier, Label::getVersionedComponentId);
+            final Map<String, RemoteProcessGroup> rpgsByVersionedId = componentsById(group, ProcessGroup::getRemoteProcessGroups,
+                RemoteProcessGroup::getIdentifier, RemoteProcessGroup::getVersionedComponentId);
+            final Map<String, ProcessGroup> childGroupsByVersionedId = componentsById(group, ProcessGroup::getProcessGroups, ProcessGroup::getIdentifier, ProcessGroup::getVersionedComponentId);
+
+            removeMissingProcessors(group, proposed, processorsByVersionedId);
+            removeMissingFunnels(group, proposed, funnelsByVersionedId);
+            removeMissingInputPorts(group, proposed, inputPortsByVersionedId);
+            removeMissingOutputPorts(group, proposed, outputPortsByVersionedId);
+            removeMissingLabels(group, proposed, labelsByVersionedId);
+            removeMissingRpg(group, proposed, rpgsByVersionedId);
+            removeMissingChildGroups(group, proposed, childGroupsByVersionedId);
+
+            // Synchronize Child Process Groups
+            synchronizeChildGroups(group, proposed, versionedParameterContexts, childGroupsByVersionedId);
+
+            synchronizeFunnels(group, proposed, funnelsByVersionedId);
+            synchronizeInputPorts(group, proposed, proposedPortFinalNames, inputPortsByVersionedId);
+            synchronizeOutputPorts(group, proposed, proposedPortFinalNames, outputPortsByVersionedId);
+            synchronizeLabels(group, proposed, labelsByVersionedId);
+            synchronizeProcessors(group, proposed, autoTerminatedRelationships, processorsByVersionedId);
+            synchronizeRemoteGroups(group, proposed, rpgsByVersionedId);
+        } finally {
+            // Make sure that we reset the connections
+            restoreConnectionDestinations(group, proposed, connectionsByVersionedId, connectionsWithTempDestination);
+            removeTemporaryFunnel(group);
+        }
+
+        // We can now add in any necessary connections, since all connectable components have now been created.
+        synchronizeConnections(group, proposed, connectionsByVersionedId);
+
+        // Once the appropriate connections have been removed, we may now update Processors' auto-terminated relationships.
+        // We cannot do this above, in the 'updateProcessor' call because if a connection is removed and changed to auto-terminated,
+        // then updating this in the updateProcessor call above would attempt to set the Relationship to being auto-terminated while a
+        // Connection for that relationship exists. This will throw an Exception.
+        autoTerminatedRelationships.forEach(ProcessorNode::setAutoTerminatedRelationships);
+
+        // All ports have now been added/removed as necessary. We can now resolve the port names.
+        updatePortsToFinalNames(proposedPortFinalNames);
+
+        // Start all components that are queued up to be started now
+        context.getComponentScheduler().resume();
+    }
+
+    private void synchronizeChildGroups(final ProcessGroup group, final VersionedProcessGroup proposed, final Map<String, VersionedParameterContext> versionedParameterContexts,
+                                        final Map<String, ProcessGroup> childGroupsByVersionedId) throws ProcessorInstantiationException {
+
+        for (final VersionedProcessGroup proposedChildGroup : proposed.getProcessGroups()) {
+            final ProcessGroup childGroup = childGroupsByVersionedId.get(proposedChildGroup.getIdentifier());
+            final VersionedFlowCoordinates childCoordinates = proposedChildGroup.getVersionedFlowCoordinates();
+
+            // if there is a nested process group that is version controlled, make sure get the param contexts that go with that snapshot
+            // instead of the ones from the parent which would have been passed in to this method
+            Map<String, VersionedParameterContext> childParameterContexts = versionedParameterContexts;
+            if (childCoordinates != null && syncOptions.isUpdateDescendantVersionedFlows()) {
+                final String childParameterContextName = proposedChildGroup.getParameterContextName();
+                if (childParameterContextName != null && !versionedParameterContexts.containsKey(childParameterContextName)) {
+                    childParameterContexts = getVersionedParameterContexts(childCoordinates);
+                } else {
+                    childParameterContexts = versionedParameterContexts;
+                }
+            }
+
+            if (childGroup == null) {
+                final ProcessGroup added = addProcessGroup(group, proposedChildGroup, context.getComponentIdGenerator(), preExistingVariables, childParameterContexts);
+                context.getFlowManager().onProcessGroupAdded(added);
+                added.findAllRemoteProcessGroups().forEach(RemoteProcessGroup::initialize);
+                LOG.info("Added {} to {}", added, group);
+            } else if (childCoordinates == null || syncOptions.isUpdateDescendantVersionedFlows()) {
+
+                final StandardProcessGroupSynchronizer sync = new StandardProcessGroupSynchronizer(context);
+                sync.setPreExistingVariables(preExistingVariables);
+                sync.setUpdatedVersionedComponentIds(updatedVersionedComponentIds);
+                final GroupSynchronizationOptions options = GroupSynchronizationOptions.Builder.from(syncOptions)
+                    .updateGroupSettings(true)
+                    .build();
+
+                sync.setSynchronizationOptions(options);
+                sync.synchronize(childGroup, proposedChildGroup, childParameterContexts);
+
+                LOG.info("Updated {}", childGroup);
+            }
+        }
+    }
+
+    private void synchronizeControllerServices(final ProcessGroup group, final VersionedProcessGroup proposed, final Map<String, ControllerServiceNode> servicesByVersionedId) {
+        // Controller Services have to be handled a bit differently than other components. This is because Processors and Controller
+        // Services may reference other Controller Services. Since we may be adding Service A, which depends on Service B, before adding
+        // Service B, we need to ensure that we create all Controller Services first and then call updateControllerService for each
+        // Controller Service. This way, we ensure that all services have been created before setting the properties. This allows us to
+        // properly obtain the correct mapping of Controller Service VersionedComponentID to Controller Service instance id.
+        final Map<ControllerServiceNode, VersionedControllerService> services = new HashMap<>();
+
+        // Add any Controller Service that does not yet exist.
+        final Map<String, ControllerServiceNode> servicesAdded = new HashMap<>();
+        for (final VersionedControllerService proposedService : proposed.getControllerServices()) {
+            ControllerServiceNode service = servicesByVersionedId.get(proposedService.getIdentifier());
+            if (service == null) {
+                service = addControllerService(group, proposedService.getIdentifier(), proposedService.getInstanceIdentifier(),
+                    proposedService.getType(), proposedService.getBundle(), context.getComponentIdGenerator());
+
+                LOG.info("Added {} to {}", service, group);
+                servicesAdded.put(proposedService.getIdentifier(), service);
+            }
+
+            services.put(service, proposedService);
+        }
+
+        // Because we don't know what order to instantiate the Controller Services, it's possible that we have two services such that Service A references Service B.
+        // If Service A happens to get created before Service B, the identifiers won't get matched up. As a result, we now iterate over all created Controller Services
+        // and update them again now that all Controller Services have been created at this level, so that the linkage can now be properly established.
+        for (final VersionedControllerService proposedService : proposed.getControllerServices()) {
+            final ControllerServiceNode addedService = servicesAdded.get(proposedService.getIdentifier());
+            if (addedService == null) {
+                continue;
+            }
+
+            updateControllerService(addedService, proposedService);
+        }
+
+        // Update all of the Controller Services to match the VersionedControllerService
+        for (final Map.Entry<ControllerServiceNode, VersionedControllerService> entry : services.entrySet()) {
+            final ControllerServiceNode service = entry.getKey();
+            final VersionedControllerService proposedService = entry.getValue();
+
+            if (updatedVersionedComponentIds.contains(proposedService.getIdentifier())) {
+                updateControllerService(service, proposedService);
+                LOG.info("Updated {}", service);
+            }
+        }
+
+        // Determine all Controller Services whose scheduled state indicate they should be enabled.
+        final Set<ControllerServiceNode> toEnable = new HashSet<>();
+        for (final Map.Entry<ControllerServiceNode, VersionedControllerService> entry : services.entrySet()) {
+            if (entry.getValue().getScheduledState() == org.apache.nifi.flow.ScheduledState.ENABLED) {
+                toEnable.add(entry.getKey());
+            }
+        }
+
+        // Perform Validation so we can enable controller services and then enable them
+        toEnable.forEach(ComponentNode::performValidation);
+
+        // Enable the services. We have to do this at the end, after creating all of them, in case one service depends on another and
+        // therefore is not valid until all have been created.
+        toEnable.forEach(service -> {
+            if (service.getState() == ControllerServiceState.DISABLED) {
+                LOG.debug("Enabling {}", service);
+                context.getControllerServiceProvider().enableControllerServicesAsync(Collections.singleton(service));
+            }
+        });
+    }
+
+    private void removeMissingConnections(final ProcessGroup group, final VersionedProcessGroup proposed, final Map<String, Connection> connectionsByVersionedId) {
+        final Set<String> connectionsRemoved = new HashSet<>(connectionsByVersionedId.keySet());
+
+        for (final VersionedConnection proposedConnection : proposed.getConnections()) {
+            connectionsRemoved.remove(proposedConnection.getIdentifier());
+        }
+
+        for (final String removedVersionedId : connectionsRemoved) {
+            final Connection connection = connectionsByVersionedId.get(removedVersionedId);
+            LOG.info("Removing {} from {}", connection, group);
+            group.removeConnection(connection);
+            context.getFlowManager().onConnectionRemoved(connection);
+        }
+    }
+
+    private void synchronizeConnections(final ProcessGroup group, final VersionedProcessGroup proposed, final Map<String, Connection> connectionsByVersionedId) {
+        // Add and update Connections
+        for (final VersionedConnection proposedConnection : proposed.getConnections()) {
+            final Connection connection = connectionsByVersionedId.get(proposedConnection.getIdentifier());
+            if (connection == null) {
+                final Connection added = addConnection(group, proposedConnection, context.getComponentIdGenerator());
+                context.getFlowManager().onConnectionAdded(added);
+                LOG.info("Added {} to {}", added, group);
+            } else if (isUpdateable(connection)) {
+                // If the connection needs to be updated, then the source and destination will already have
+                // been stopped (else, the validation above would fail). So if the source or the destination is running,
+                // then we know that we don't need to update the connection.
+                updateConnection(connection, proposedConnection);
+                LOG.info("Updated {}", connection);
+            }
+        }
+    }
+
+    private Set<String> updateConnectionDestinations(final ProcessGroup group, final VersionedProcessGroup proposed, final Map<String, Connection> connectionsByVersionedId) {
+
+        final Set<String> connectionsWithTempDestination = new HashSet<>();
+        for (final VersionedConnection proposedConnection : proposed.getConnections()) {
+            final Connection connection = connectionsByVersionedId.get(proposedConnection.getIdentifier());
+            if (connection == null) {
+                continue;
+            }
+
+            // If the Connection's destination didn't change, nothing to do
+            final String destinationVersionId = connection.getDestination().getVersionedComponentId().orElse(null);
+            final String proposedDestinationId = proposedConnection.getDestination().getId();
+            if (Objects.equals(destinationVersionId, proposedDestinationId)) {
+                continue;
+            }
+
+            // Find the destination of the connection. If the destination doesn't yet exist (because it's part of the proposed Process Group but not yet added),
+            // we will set the destination to a temporary destination. Then, after adding components, we will update the destinations again.
+            Connectable newDestination = getConnectable(group, proposedConnection.getDestination());
+            if (newDestination == null) {
+                final Funnel temporaryDestination = getTemporaryFunnel(connection.getProcessGroup());
+                LOG.debug("Updated Connection {} to have a temporary destination of {}", connection, temporaryDestination);
+                newDestination = temporaryDestination;
+                connectionsWithTempDestination.add(proposedConnection.getIdentifier());
+            }
+
+            connection.setDestination(newDestination);
+        }
+
+        return connectionsWithTempDestination;
+    }
+
+    private Funnel getTemporaryFunnel(final ProcessGroup group) {
+        final String tempFunnelId = group.getIdentifier() + TEMP_FUNNEL_ID_SUFFIX;
+        Funnel temporaryFunnel = context.getFlowManager().getFunnel(tempFunnelId);
+        if (temporaryFunnel == null) {
+            temporaryFunnel = context.getFlowManager().createFunnel(tempFunnelId);
+            temporaryFunnel.setPosition(new Position(0, 0));
+            group.addFunnel(temporaryFunnel, false);
+        }
+
+        return temporaryFunnel;
+    }
+
+    private void restoreConnectionDestinations(final ProcessGroup group, final VersionedProcessGroup proposed, final Map<String, Connection> connectionsByVersionedId,
+                                               final Set<String> connectionsWithTempDestination) {
+        if (connectionsWithTempDestination.isEmpty()) {
+            LOG.debug("No connections with temporary destinations for {}", group);
+            return;
+        }
+
+        final Map<String, VersionedConnection> versionedConnectionsById = proposed.getConnections().stream()
+            .collect(Collectors.toMap(VersionedConnection::getIdentifier, Function.identity()));
+
+        for (final String connectionId : connectionsWithTempDestination) {
+            final Connection connection = connectionsByVersionedId.get(connectionId);
+            final VersionedConnection versionedConnection = versionedConnectionsById.get(connectionId);
+
+            final Connectable newDestination = getConnectable(group, versionedConnection.getDestination());
+            if (newDestination != null) {
+                LOG.debug("Updated Connection {} from its temporary destination to its correct destination of {}", connection, newDestination);
+                connection.setDestination(newDestination);
+            }
+        }
+    }
+
+    private void removeTemporaryFunnel(final ProcessGroup group) {
+        final String tempFunnelId = group.getIdentifier() + TEMP_FUNNEL_ID_SUFFIX;
+        final Funnel temporaryFunnel = context.getFlowManager().getFunnel(tempFunnelId);
+        if (temporaryFunnel == null) {
+            LOG.debug("No temporary funnel to remove for {}", group);
+            return;
+        }
+
+        if (temporaryFunnel.getIncomingConnections().isEmpty()) {
+            LOG.debug("Updated all temporary connections for {}. Removing Temporary funnel from flow", group);
+            group.removeFunnel(temporaryFunnel);
+        } else {
+            LOG.warn("The temporary funnel {} for {} still has {} connections. It cannot be removed.", temporaryFunnel, group, temporaryFunnel.getIncomingConnections().size());
+        }
+    }
+
+    private <T extends Connectable> Map<String, T> componentsById(final ProcessGroup group, final Function<ProcessGroup, Collection<T>> retrieveComponents) {
+        return retrieveComponents.apply(group).stream()
+            .collect(Collectors.toMap(component -> component.getVersionedComponentId().orElse(
+                NiFiRegistryFlowMapper.generateVersionedComponentId(component.getIdentifier())), Function.identity()));
+    }
+
+
+    private <T> Map<String, T> componentsById(final ProcessGroup group, final Function<ProcessGroup, Collection<T>> retrieveComponents,
+                                              final Function<T, String> retrieveId, final Function<T, Optional<String>> retrieveVersionedComponentId) {
+
+        return retrieveComponents.apply(group).stream()
+            .collect(Collectors.toMap(component -> retrieveVersionedComponentId.apply(component).orElse(
+                NiFiRegistryFlowMapper.generateVersionedComponentId(retrieveId.apply(component))), Function.identity()));
+    }
+
+
+    private void synchronizeFunnels(final ProcessGroup group, final VersionedProcessGroup proposed, final Map<String, Funnel> funnelsByVersionedId) {
+        for (final VersionedFunnel proposedFunnel : proposed.getFunnels()) {
+            final Funnel funnel = funnelsByVersionedId.get(proposedFunnel.getIdentifier());
+            if (funnel == null) {
+                final Funnel added = addFunnel(group, proposedFunnel, context.getComponentIdGenerator());
+                context.getFlowManager().onFunnelAdded(added);
+                LOG.info("Added {} to {}", added, group);
+            } else if (updatedVersionedComponentIds.contains(proposedFunnel.getIdentifier())) {
+                updateFunnel(funnel, proposedFunnel);
+                LOG.info("Updated {}", funnel);
+            } else {
+                funnel.setPosition(new Position(proposedFunnel.getPosition().getX(), proposedFunnel.getPosition().getY()));
+            }
+        }
+    }
+
+    private void synchronizeInputPorts(final ProcessGroup group, final VersionedProcessGroup proposed, final Map<Port, String> proposedPortFinalNames,
+                                                              final Map<String, Port> inputPortsByVersionedId) {
+        for (final VersionedPort proposedPort : proposed.getInputPorts()) {
+            final Port port = inputPortsByVersionedId.get(proposedPort.getIdentifier());
+            if (port == null) {
+                final String temporaryName = generateTemporaryPortName(proposedPort);
+                final Port added = addInputPort(group, proposedPort, context.getComponentIdGenerator(), temporaryName);
+                proposedPortFinalNames.put(added, proposedPort.getName());
+                context.getFlowManager().onInputPortAdded(added);
+                LOG.info("Added {} to {}", added, group);
+            } else if (updatedVersionedComponentIds.contains(proposedPort.getIdentifier())) {
+                final String temporaryName = generateTemporaryPortName(proposedPort);
+                proposedPortFinalNames.put(port, proposedPort.getName());
+                updatePort(port, proposedPort, temporaryName);
+                LOG.info("Updated {}", port);
+            } else {
+                port.setPosition(new Position(proposedPort.getPosition().getX(), proposedPort.getPosition().getY()));
+            }
+        }
+    }
+
+    private void synchronizeOutputPorts(final ProcessGroup group, final VersionedProcessGroup proposed, final Map<Port, String> proposedPortFinalNames,
+                                        final Map<String, Port> outputPortsByVersionedId) {
+
+        for (final VersionedPort proposedPort : proposed.getOutputPorts()) {
+            final Port port = outputPortsByVersionedId.get(proposedPort.getIdentifier());
+            if (port == null) {
+                final String temporaryName = generateTemporaryPortName(proposedPort);
+                final Port added = addOutputPort(group, proposedPort, context.getComponentIdGenerator(), temporaryName);
+                proposedPortFinalNames.put(added, proposedPort.getName());
+                context.getFlowManager().onOutputPortAdded(added);
+                LOG.info("Added {} to {}", added, group);
+            } else if (updatedVersionedComponentIds.contains(proposedPort.getIdentifier())) {
+                final String temporaryName = generateTemporaryPortName(proposedPort);
+                proposedPortFinalNames.put(port, proposedPort.getName());
+                updatePort(port, proposedPort, temporaryName);
+                LOG.info("Updated {}", port);
+            } else {
+                port.setPosition(new Position(proposedPort.getPosition().getX(), proposedPort.getPosition().getY()));
+            }
+        }
+    }
+
+    private void updatePortsToFinalNames(final Map<Port, String> proposedPortFinalNames) {
+        // Now that all input/output ports have been removed, we should be able to update
+        // all ports to the final name that was proposed in the new flow version.
+        for (final Map.Entry<Port, String> portAndFinalName : proposedPortFinalNames.entrySet()) {
+            final Port port = portAndFinalName.getKey();
+            final String finalName = portAndFinalName.getValue();
+            LOG.info("Updating {} to replace temporary name with final name", port);
+
+            // For public ports we need to consider if another public port exists somewhere else in the flow with the
+            // same name, and if so then rename the incoming port so the flow can still be imported
+            if (port instanceof PublicPort) {
+                final PublicPort publicPort = (PublicPort) port;
+                final String publicPortFinalName = getPublicPortFinalName(publicPort, finalName);
+                updatePortToSetFinalName(publicPort, publicPortFinalName);
+            } else {
+                updatePortToSetFinalName(port, finalName);
+            }
+        }
+    }
+
+    private void synchronizeLabels(final ProcessGroup group, final VersionedProcessGroup proposed, final Map<String, Label> labelsByVersionedId) {
+        for (final VersionedLabel proposedLabel : proposed.getLabels()) {
+            final Label label = labelsByVersionedId.get(proposedLabel.getIdentifier());
+            if (label == null) {
+                final Label added = addLabel(group, proposedLabel, context.getComponentIdGenerator());
+                LOG.info("Added {} to {}", added, group);
+            } else if (updatedVersionedComponentIds.contains(proposedLabel.getIdentifier())) {
+                updateLabel(label, proposedLabel);
+                LOG.info("Updated {}", label);
+            } else {
+                label.setPosition(new Position(proposedLabel.getPosition().getX(), proposedLabel.getPosition().getY()));
+            }
+        }
+    }
+
+    private void removeMissingProcessors(final ProcessGroup group, final VersionedProcessGroup proposed, final Map<String, ProcessorNode> processorsByVersionedId) {
+        removeMissingComponents(group, proposed, processorsByVersionedId, VersionedProcessGroup::getProcessors, ProcessGroup::removeProcessor);
+    }
+
+    private void removeMissingInputPorts(final ProcessGroup group, final VersionedProcessGroup proposed, final Map<String, Port> portsByVersionedId) {
+        removeMissingComponents(group, proposed, portsByVersionedId, VersionedProcessGroup::getInputPorts, ProcessGroup::removeInputPort);
+    }
+
+    private void removeMissingOutputPorts(final ProcessGroup group, final VersionedProcessGroup proposed, final Map<String, Port> portsByVersionedId) {
+        removeMissingComponents(group, proposed, portsByVersionedId, VersionedProcessGroup::getOutputPorts, ProcessGroup::removeOutputPort);
+    }
+
+    private void removeMissingLabels(final ProcessGroup group, final VersionedProcessGroup proposed, final Map<String, Label> labelsByVersionedId) {
+        removeMissingComponents(group, proposed, labelsByVersionedId, VersionedProcessGroup::getLabels, ProcessGroup::removeLabel);
+    }
+
+    private void removeMissingFunnels(final ProcessGroup group, final VersionedProcessGroup proposed, final Map<String, Funnel> funnelsByVersionedId) {
+        removeMissingComponents(group, proposed, funnelsByVersionedId, VersionedProcessGroup::getFunnels, (removalGroup, funnelToRemove) -> {
+            // Skip our temporary funnel
+            if (funnelToRemove.getIdentifier().equals(removalGroup.getIdentifier() + TEMP_FUNNEL_ID_SUFFIX)) {
+                return;
+            }
+
+            removalGroup.removeFunnel(funnelToRemove);
+        });
+    }
+
+    private void removeMissingRpg(final ProcessGroup group, final VersionedProcessGroup proposed, final Map<String, RemoteProcessGroup> rpgsByVersionedId) {
+        removeMissingComponents(group, proposed, rpgsByVersionedId, VersionedProcessGroup::getRemoteProcessGroups, ProcessGroup::removeRemoteProcessGroup);
+    }
+
+    private void removeMissingControllerServices(final ProcessGroup group, final VersionedProcessGroup proposed, final Map<String, ControllerServiceNode> servicesByVersionedId) {
+        final BiConsumer<ProcessGroup, ControllerServiceNode> componentRemoval = (grp, service) -> context.getControllerServiceProvider().removeControllerService(service);
+        removeMissingComponents(group, proposed, servicesByVersionedId, VersionedProcessGroup::getControllerServices, componentRemoval);
+    }
+
+    private void removeMissingChildGroups(final ProcessGroup group, final VersionedProcessGroup proposed, final Map<String, ProcessGroup> groupsByVersionedId) {
+        removeMissingComponents(group, proposed, groupsByVersionedId, VersionedProcessGroup::getProcessGroups,
+            (procGroup, childGroup) -> {
+                // We cannot remove a Process Group unless it is empty. At this point, we've already removed
+                // all Processors, Input Ports, etc. that are no longer needed. However, we have not removed all
+                // Process Groups. We may have a situation where we have nested Process Groups, each one consisting
+                // now of only other Process Groups that can be removed, such as A -> B -> C -> D.
+                // Each of these is a Process Group that contains only other (otherwise empty) process groups.
+                // To accomplish this, we need to use a depth-first approach, removing the inner-most group (D),
+                // then C, then B, and finally A.
+                if (!childGroup.isEmpty()) {
+                    purgeChildGroupOfEmptyChildren(childGroup);
+                }
+
+                procGroup.removeProcessGroup(childGroup);
+            });
+    }
+
+    private void purgeChildGroupOfEmptyChildren(final ProcessGroup group) {
+        for (final ProcessGroup child : group.getProcessGroups()) {
+            purgeChildGroupOfEmptyChildren(child);
+
+            if (child.isEmpty()) {
+                group.removeProcessGroup(child);
+            }
+        }
+    }
+
+    private <C, V extends VersionedComponent> void removeMissingComponents(final ProcessGroup group, final VersionedProcessGroup proposed, final Map<String, C> componentsById,
+                                             final Function<VersionedProcessGroup, Collection<V>> getVersionedComponents, final BiConsumer<ProcessGroup, C> removeComponent) {
+
+        // Determine the ID's of the components to remove. To do this, we get the ID's of all components in the Process Group,
+        // and then remove from that the ID's of the components in the proposed group. That leaves us with the ID's of components
+        // that exist currently that are not in the proposed flow.
+        final Set<String> idsOfComponentsToRemove = new HashSet<>(componentsById.keySet());
+        for (final V versionedComponent : getVersionedComponents.apply(proposed)) {
+            idsOfComponentsToRemove.remove(versionedComponent.getIdentifier());
+        }
+
+        // Remove any of those components
+        for (final String idToRemove : idsOfComponentsToRemove) {
+            final C toRemove = componentsById.get(idToRemove);
+            LOG.info("Removing {} from {}", toRemove, group);
+            removeComponent.accept(group, toRemove);
+        }
+    }
+
+
+    private void synchronizeProcessors(final ProcessGroup group, final VersionedProcessGroup proposed, final Map<ProcessorNode, Set<Relationship>> autoTerminatedRelationships,
+                                                                       final Map<String, ProcessorNode> processorsByVersionedId) throws ProcessorInstantiationException {
+        for (final VersionedProcessor proposedProcessor : proposed.getProcessors()) {
+            final ProcessorNode processor = processorsByVersionedId.get(proposedProcessor.getIdentifier());
+            if (processor == null) {
+                final ProcessorNode added = addProcessor(group, proposedProcessor, context.getComponentIdGenerator());
+                context.getFlowManager().onProcessorAdded(added);
+
+                final Set<Relationship> proposedAutoTerminated =
+                    proposedProcessor.getAutoTerminatedRelationships() == null ? Collections.emptySet() : proposedProcessor.getAutoTerminatedRelationships().stream()
+                        .map(added::getRelationship)
+                        .collect(Collectors.toSet());
+                autoTerminatedRelationships.put(added, proposedAutoTerminated);
+                LOG.info("Added {} to {}", added, group);
+            } else if (updatedVersionedComponentIds.contains(proposedProcessor.getIdentifier())) {
+                updateProcessor(processor, proposedProcessor);
+
+                final Set<Relationship> proposedAutoTerminated =
+                    proposedProcessor.getAutoTerminatedRelationships() == null ? Collections.emptySet() : proposedProcessor.getAutoTerminatedRelationships().stream()
+                        .map(processor::getRelationship)
+                        .collect(Collectors.toSet());
+
+                if (!processor.getAutoTerminatedRelationships().equals(proposedAutoTerminated)) {
+                    autoTerminatedRelationships.put(processor, proposedAutoTerminated);
+                }
+
+                LOG.info("Updated {}", processor);
+            } else {
+                processor.setPosition(new Position(proposedProcessor.getPosition().getX(), proposedProcessor.getPosition().getY()));
+            }
+        }
+    }
+
+    private void synchronizeRemoteGroups(final ProcessGroup group, final VersionedProcessGroup proposed, final Map<String, RemoteProcessGroup> rpgsByVersionedId) {
+        for (final VersionedRemoteProcessGroup proposedRpg : proposed.getRemoteProcessGroups()) {
+            final RemoteProcessGroup rpg = rpgsByVersionedId.get(proposedRpg.getIdentifier());
+            if (rpg == null) {
+                final RemoteProcessGroup added = addRemoteProcessGroup(group, proposedRpg, context.getComponentIdGenerator());
+                LOG.info("Added {} to {}", added, group);
+            } else if (updatedVersionedComponentIds.contains(proposedRpg.getIdentifier())) {
+                updateRemoteProcessGroup(rpg, proposedRpg, context.getComponentIdGenerator());
+                LOG.info("Updated {}", rpg);
+            } else {
+                rpg.setPosition(new Position(proposedRpg.getPosition().getX(), proposedRpg.getPosition().getY()));
+            }
+        }
+    }
+
+    @Override
+    public void verifyCanSynchronize(final ProcessGroup group, final VersionedProcessGroup flowContents, final boolean verifyConnectionRemoval) {
+        // Ensure no deleted child process groups contain templates and optionally no deleted connections contain data
+        // in their queue. Note that this check enforces ancestry among the group components to avoid a scenario where
+        // a component is matched by id, but it does not exist in the same hierarchy and thus will be removed and
+        // re-added when the update is performed
+        verifyCanRemoveMissingComponents(group, flowContents, verifyConnectionRemoval);
+
+        // Determine which input ports were removed from this process group
+        final Map<String, Port> removedInputPortsByVersionId = new HashMap<>();
+        group.getInputPorts()
+            .forEach(port -> removedInputPortsByVersionId.put(port.getVersionedComponentId().orElse(
+                NiFiRegistryFlowMapper.generateVersionedComponentId(port.getIdentifier())), port));
+
+        flowContents.getInputPorts().stream()
+            .map(VersionedPort::getIdentifier)
+            .forEach(removedInputPortsByVersionId::remove);
+
+        // Ensure that there are no incoming connections for any Input Port that was removed.
+        for (final Port inputPort : removedInputPortsByVersionId.values()) {
+            final List<Connection> incomingConnections = inputPort.getIncomingConnections();
+            if (!incomingConnections.isEmpty()) {
+                throw new IllegalStateException(group + " cannot be updated to the proposed flow because the proposed flow "
+                    + "does not contain the Input Port " + inputPort + " and the Input Port currently has an incoming connection");
+            }
+        }
+
+        // Determine which output ports were removed from this process group
+        final Map<String, Port> removedOutputPortsByVersionId = new HashMap<>();
+        group.getOutputPorts()
+            .forEach(port -> removedOutputPortsByVersionId.put(port.getVersionedComponentId().orElse(
+                NiFiRegistryFlowMapper.generateVersionedComponentId(port.getIdentifier())), port));
+
+        flowContents.getOutputPorts().stream()
+            .map(VersionedPort::getIdentifier)
+            .forEach(removedOutputPortsByVersionId::remove);
+
+        // Ensure that there are no outgoing connections for any Output Port that was removed.
+        for (final Port outputPort : removedOutputPortsByVersionId.values()) {
+            final Set<Connection> outgoingConnections = outputPort.getConnections();
+            if (!outgoingConnections.isEmpty()) {
+                throw new IllegalStateException(group + " cannot be updated to the proposed flow because the proposed flow "
+                    + "does not contain the Output Port " + outputPort + " and the Output Port currently has an outgoing connection");
+            }
+        }
+
+        // Ensure that all Processors are instantiable
+        final Map<String, VersionedProcessor> proposedProcessors = new HashMap<>();
+        findAllProcessors(flowContents, proposedProcessors);
+
+        group.findAllProcessors()
+            .forEach(proc -> proposedProcessors.remove(proc.getVersionedComponentId().orElse(
+                NiFiRegistryFlowMapper.generateVersionedComponentId(proc.getIdentifier()))));
+
+        for (final VersionedProcessor processorToAdd : proposedProcessors.values()) {
+            final String processorToAddClass = processorToAdd.getType();
+            final BundleCoordinate processorToAddCoordinate = toCoordinate(processorToAdd.getBundle());
+
+            // Get the exact bundle requested, if it exists.
+            final Bundle bundle = processorToAdd.getBundle();
+            final BundleCoordinate coordinate = new BundleCoordinate(bundle.getGroup(), bundle.getArtifact(), bundle.getVersion());
+            final org.apache.nifi.bundle.Bundle resolved = context.getExtensionManager().getBundle(coordinate);
+
+            if (resolved == null) {
+                // Could not resolve the bundle explicitly. Check for possible bundles.
+                final List<org.apache.nifi.bundle.Bundle> possibleBundles = context.getExtensionManager().getBundles(processorToAddClass);
+                final boolean bundleExists = possibleBundles.stream()
+                    .anyMatch(b -> processorToAddCoordinate.equals(b.getBundleDetails().getCoordinate()));
+
+                if (!bundleExists && possibleBundles.size() != 1) {
+                    LOG.warn("Unknown bundle {} for processor type {} - will use Ghosted component instead", processorToAddCoordinate, processorToAddClass);
+                }
+            }
+        }
+
+        // Ensure that all Controller Services are instantiable
+        final Map<String, VersionedControllerService> proposedServices = new HashMap<>();
+        findAllControllerServices(flowContents, proposedServices);
+
+        group.findAllControllerServices()
+            .forEach(service -> proposedServices.remove(service.getVersionedComponentId().orElse(
+                NiFiRegistryFlowMapper.generateVersionedComponentId(service.getIdentifier()))));
+
+        for (final VersionedControllerService serviceToAdd : proposedServices.values()) {
+            final String serviceToAddClass = serviceToAdd.getType();
+            final BundleCoordinate serviceToAddCoordinate = toCoordinate(serviceToAdd.getBundle());
+
+            final org.apache.nifi.bundle.Bundle resolved = context.getExtensionManager().getBundle(serviceToAddCoordinate);
+            if (resolved == null) {
+                final List<org.apache.nifi.bundle.Bundle> possibleBundles = context.getExtensionManager().getBundles(serviceToAddClass);
+                final boolean bundleExists = possibleBundles.stream()
+                    .anyMatch(b -> serviceToAddCoordinate.equals(b.getBundleDetails().getCoordinate()));
+
+                if (!bundleExists && possibleBundles.size() != 1) {
+                    LOG.warn("Unknown bundle {} for processor type {} - will use Ghosted component instead", serviceToAddCoordinate, serviceToAddClass);
+                }
+            }
+        }
+
+        // Ensure that all Prioritizers are instantiable and that any load balancing configuration is correct
+        // Enforcing ancestry on connection matching here is not important because all we're interested in is locating
+        // new prioritizers and load balance strategy types so if a matching connection existed anywhere in the current
+        // flow, then its prioritizer and load balance strategy are already validated
+        final Map<String, VersionedConnection> proposedConnections = new HashMap<>();
+        findAllConnections(flowContents, proposedConnections);
+
+        group.findAllConnections()
+            .forEach(conn -> proposedConnections.remove(conn.getVersionedComponentId().orElse(
+                NiFiRegistryFlowMapper.generateVersionedComponentId(conn.getIdentifier()))));
+
+        for (final VersionedConnection connectionToAdd : proposedConnections.values()) {
+            if (connectionToAdd.getPrioritizers() != null) {
+                for (final String prioritizerType : connectionToAdd.getPrioritizers()) {
+                    try {
+                        context.getFlowManager().createPrioritizer(prioritizerType);
+                    } catch (Exception e) {
+                        throw new IllegalArgumentException("Unable to create Prioritizer of type " + prioritizerType, e);
+                    }
+                }
+            }
+
+            final String loadBalanceStrategyName = connectionToAdd.getLoadBalanceStrategy();
+            if (loadBalanceStrategyName != null) {
+                try {
+                    LoadBalanceStrategy.valueOf(loadBalanceStrategyName);
+                } catch (final IllegalArgumentException iae) {
+                    throw new IllegalArgumentException("Unable to create Connection with Load Balance Strategy of '" + loadBalanceStrategyName
+                        + "' because this is not a known Load Balance Strategy");
+                }
+            }
+        }
+    }
+
+    private ProcessGroup addProcessGroup(final ProcessGroup destination, final VersionedProcessGroup proposed, final ComponentIdGenerator componentIdGenerator, final Set<String> variablesToSkip,
+                                         final Map<String, VersionedParameterContext> versionedParameterContexts) throws ProcessorInstantiationException {
+        final String id = componentIdGenerator.generateUuid(proposed.getIdentifier(), proposed.getInstanceIdentifier(), destination.getIdentifier());
+        final ProcessGroup group = context.getFlowManager().createProcessGroup(id);
+        group.setVersionedComponentId(proposed.getIdentifier());
+        group.setParent(destination);
+        group.setName(proposed.getName());
+
+        destination.addProcessGroup(group);
+
+        final StandardProcessGroupSynchronizer sync = new StandardProcessGroupSynchronizer(context);
+        sync.setPreExistingVariables(variablesToSkip);
+        sync.setUpdatedVersionedComponentIds(updatedVersionedComponentIds);
+
+        final GroupSynchronizationOptions options = GroupSynchronizationOptions.Builder.from(syncOptions)
+            .updateGroupSettings(true)
+            .build();
+        sync.setSynchronizationOptions(options);
+        sync.synchronize(group, proposed, versionedParameterContexts);
+
+        return group;
+    }
+
+    private ControllerServiceNode addControllerService(final ProcessGroup destination, final String versionedId, final String instanceId, final String type, final Bundle bundle,
+                                                                         final ComponentIdGenerator componentIdGenerator) {
+        final String id = componentIdGenerator.generateUuid(versionedId, instanceId, destination.getIdentifier());
+        LOG.debug("Adding Controller Service with ID {} of type {}", id, type);
+
+        final BundleCoordinate coordinate = toCoordinate(bundle);
+        final boolean firstTimeAdded = true;
+        final Set<URL> additionalUrls = Collections.emptySet();
+
+        final ControllerServiceNode newService = context.getFlowManager().createControllerService(type, id, coordinate, additionalUrls, firstTimeAdded, true, null);
+        newService.setVersionedComponentId(versionedId);
+
+        destination.addControllerService(newService);
+
+        return newService;
+    }
+
+
+    private void updateControllerService(final ControllerServiceNode service, final VersionedControllerService proposed) {
+        LOG.debug("Updating {}", service);
+
+        service.pauseValidationTrigger();
+        try {
+            service.setAnnotationData(proposed.getAnnotationData());
+            service.setComments(proposed.getComments());
+            service.setName(proposed.getName());
+
+            final Map<String, String> properties = populatePropertiesMap(service, proposed.getProperties(), proposed.getPropertyDescriptors(), service.getProcessGroup());
+            service.setProperties(properties, true);
+
+            if (!isEqual(service.getBundleCoordinate(), proposed.getBundle())) {
+                final BundleCoordinate newBundleCoordinate = toCoordinate(proposed.getBundle());
+                final List<PropertyDescriptor> descriptors = new ArrayList<>(service.getRawPropertyValues().keySet());
+                final Set<URL> additionalUrls = service.getAdditionalClasspathResources(descriptors);
+                context.getReloadComponent().reload(service, proposed.getType(), newBundleCoordinate, additionalUrls);
+            }
+        } finally {
+            service.resumeValidationTrigger();
+        }
+    }
+
+    private Map<String, String> populatePropertiesMap(final ComponentNode componentNode, final Map<String, String> proposedProperties,
+                                                      final Map<String, VersionedPropertyDescriptor> proposedDescriptors, final ProcessGroup group) {
+
+        // Explicitly set all existing properties to null, except for sensitive properties, so that if there isn't an entry in the proposedProperties
+        // it will get removed from the processor. We don't do this for sensitive properties because when we retrieve the VersionedProcessGroup from registry,
+        // any sensitive properties will already have been removed, and we don't want to clear those values, or else we'd always clear sensitive values.
+        final Map<String, String> fullPropertyMap = new HashMap<>();
+        for (final PropertyDescriptor property : componentNode.getRawPropertyValues().keySet()) {
+            if (!property.isSensitive()) {
+                fullPropertyMap.put(property.getName(), null);
+            }
+        }
+
+        if (proposedProperties != null) {
+            // Build a Set of all properties that are included in either the currently configured property values or the proposed values.
+            final Set<String> updatedPropertyNames = new HashSet<>(proposedProperties.keySet());
+            componentNode.getProperties().keySet().stream()
+                .map(PropertyDescriptor::getName)
+                .forEach(updatedPropertyNames::add);
+
+            for (final String propertyName : updatedPropertyNames) {
+                final VersionedPropertyDescriptor descriptor = proposedDescriptors.get(propertyName);
+
+                String value;
+                if (descriptor != null && descriptor.getIdentifiesControllerService()) {
+
+                    // Need to determine if the component's property descriptor for this service is already set to an id
+                    // of an existing service that is outside the current processor group, and if it is we want to leave
+                    // the property set to that value
+                    String existingExternalServiceId = null;
+                    final PropertyDescriptor componentDescriptor = componentNode.getPropertyDescriptor(propertyName);
+                    if (componentDescriptor != null) {
+                        final String componentDescriptorValue = componentNode.getEffectivePropertyValue(componentDescriptor);
+                        if (componentDescriptorValue != null) {
+                            final ProcessGroup parentGroup = group.getParent();
+                            if (parentGroup != null) {
+                                final ControllerServiceNode serviceNode = parentGroup.findControllerService(componentDescriptorValue, false, true);
+                                if (serviceNode != null) {
+                                    existingExternalServiceId = componentDescriptorValue;
+                                }
+                            }
+                        }
+                    }
+
+                    // If the component's property descriptor is not already set to an id of an existing external service,
+                    // then we need to take the Versioned Component ID and resolve this to the instance ID of the service
+                    if (existingExternalServiceId == null) {
+                        final String serviceVersionedComponentId = proposedProperties.get(propertyName);
+                        String instanceId = getServiceInstanceId(serviceVersionedComponentId, group);
+                        value = instanceId == null ? serviceVersionedComponentId : instanceId;
+                    } else {
+                        value = existingExternalServiceId;
+                    }
+
+                } else {
+                    value = proposedProperties.get(propertyName);
+                }
+
+                // skip any sensitive properties that are not populated so we can retain whatever is currently set. We do this because sensitive properties are not stored in the registry
+                // unless the value is a reference to a Parameter. If the value in the registry is null, it indicates that the sensitive value was removed, so we want to keep the currently
+                // populated value. The exception to this rule is if the currently configured value is a Parameter Reference and the Versioned Flow is empty. In this case, it implies
+                // that the Versioned Flow has changed from a Parameter Reference to an explicit value. In this case, we do in fact want to change the value of the Sensitive Property from
+                // the current parameter reference to an unset value.
+                final boolean sensitive = componentNode.getPropertyDescriptor(propertyName).isSensitive();
+                if (sensitive && value == null) {
+                    final PropertyConfiguration propertyConfiguration = componentNode.getProperty(componentNode.getPropertyDescriptor(propertyName));
+                    if (propertyConfiguration == null) {
+                        continue;
+                    }
+
+                    // No parameter references. Property currently is set to an explicit value. We don't want to change it.
+                    if (propertyConfiguration.getParameterReferences().isEmpty()) {
+                        continue;
+                    }
+
+                    // Once we reach this point, the property is configured to reference a Parameter, and the value in the Versioned Flow is an explicit value,
+                    // so we want to continue on and update the value to null.
+                }
+
+                value = decrypt(value, syncOptions.getPropertyDecryptor());
+                fullPropertyMap.put(propertyName, value);
+            }
+        }
+
+        return fullPropertyMap;
+    }
+
+    private static String decrypt(final String value, final PropertyDecryptor decryptor) {
+        if (value == null) {
+            return null;
+        }
+        if (!value.startsWith(ENC_PREFIX)) {
+            return value;
+        }
+        if (!value.endsWith(ENC_SUFFIX)) {
+            return value;
+        }
+
+        try {
+            return decryptor.decrypt(value.substring(ENC_PREFIX.length(), value.length() - ENC_SUFFIX.length()));
+        } catch (EncryptionException e) {
+            final String moreDescriptiveMessage = "There was a problem decrypting a sensitive flow configuration value. " +
+                "Check that the nifi.sensitive.props.key value in nifi.properties matches the value used to encrypt the flow.xml.gz file";
+            throw new EncryptionException(moreDescriptiveMessage, e);
+        }
+    }
+
+
+    private void updateParameterContext(final ProcessGroup group, final VersionedProcessGroup proposed, final Map<String, VersionedParameterContext> versionedParameterContexts,
+                                        final ComponentIdGenerator componentIdGenerator) {
+        // Update the Parameter Context
+        final ParameterContext currentParamContext = group.getParameterContext();
+        final String proposedParameterContextName = proposed.getParameterContextName();
+        if (proposedParameterContextName == null && currentParamContext != null) {
+            group.setParameterContext(null);
+        } else if (proposedParameterContextName != null) {
+            final VersionedParameterContext versionedParameterContext = versionedParameterContexts.get(proposedParameterContextName);
+
+            if (currentParamContext == null) {
+                // Create a new Parameter Context based on the parameters provided
+
+                // Protect against NPE in the event somehow the proposed name is not in the set of contexts
+                if (versionedParameterContext == null) {
+                    final String paramContextNames = StringUtils.join(versionedParameterContexts.keySet());
+                    throw new IllegalStateException("Proposed parameter context name '" + proposedParameterContextName
+                        + "' does not exist in set of available parameter contexts [" + paramContextNames + "]");
+                }
+
+                final ParameterContext contextByName = getParameterContextByName(versionedParameterContext.getName());
+                final ParameterContext selectedParameterContext;
+                if (contextByName == null) {
+                    final String parameterContextId = componentIdGenerator.generateUuid(versionedParameterContext.getName(), versionedParameterContext.getName(), versionedParameterContext.getName());
+                    selectedParameterContext = createParameterContext(versionedParameterContext, parameterContextId, versionedParameterContexts);
+                } else {
+                    selectedParameterContext = contextByName;
+                    addMissingConfiguration(versionedParameterContext, selectedParameterContext, versionedParameterContexts);
+                }
+
+                group.setParameterContext(selectedParameterContext);
+            } else {
+                // Update the current Parameter Context so that it has any Parameters included in the proposed context
+                addMissingConfiguration(versionedParameterContext, currentParamContext, versionedParameterContexts);
+            }
+        }
+    }
+
+    private void updateVariableRegistry(final ProcessGroup group, final VersionedProcessGroup proposed) {
+        // Determine which variables have been added/removed and add/remove them from this group's variable registry.
+        // We don't worry about if a variable value has changed, because variables are designed to be 'environment specific.'
+        // As a result, once imported, we won't update variables to match the remote flow, but we will add any missing variables
+        // and remove any variables that are no longer part of the remote flow.
+        final Map<String, String> existingVariableMap = new HashMap<>();
+        group.getVariableRegistry().getVariableMap().forEach((descriptor, value) -> existingVariableMap.put(descriptor.getName(), value));
+
+        final Map<String, String> updatedVariableMap = new HashMap<>();
+
+        // If any new variables exist in the proposed flow, add those to the variable registry.
+        for (final Map.Entry<String, String> entry : proposed.getVariables().entrySet()) {
+            final String variableName = entry.getKey();
+            final String proposedValue = entry.getValue();
+            final String existingValue = existingVariableMap.get(variableName);
+            final boolean alreadyAccessible = existingVariableMap.containsKey(variableName) || preExistingVariables.contains(variableName);
+            final boolean newVariable = !alreadyAccessible;
+
+            if (newVariable || (syncOptions.isUpdateExistingVariables() && !Objects.equals(proposedValue, existingValue))) {
+                updatedVariableMap.put(variableName, proposedValue);
+            }
+        }
+
+        group.setVariables(updatedVariableMap);
+    }
+
+    private String getPublicPortFinalName(final PublicPort publicPort, final String proposedFinalName) {
+        final Optional<Port> existingPublicPort;
+        if (TransferDirection.RECEIVE == publicPort.getDirection()) {
+            existingPublicPort = context.getFlowManager().getPublicInputPort(proposedFinalName);
+        } else {
+            existingPublicPort = context.getFlowManager().getPublicOutputPort(proposedFinalName);
+        }
+
+        if (existingPublicPort.isPresent() && !existingPublicPort.get().getIdentifier().equals(publicPort.getIdentifier())) {
+            return getPublicPortFinalName(publicPort, "Copy of " + proposedFinalName);
+        } else {
+            return proposedFinalName;
+        }
+    }
+
+    private ParameterContext getParameterContextByName(final String contextName) {
+        return context.getFlowManager().getParameterContextManager().getParameterContexts().stream()
+            .filter(context -> context.getName().equals(contextName))
+            .findAny()
+            .orElse(null);
+    }
+
+    private ParameterContext createParameterContextWithoutReferences(final VersionedParameterContext versionedParameterContext) {
+        final ParameterContext existing = context.getFlowManager().getParameterContextManager().getParameterContextNameMapping().get(versionedParameterContext.getName());
+        if (existing != null) {
+            return existing;
+        }
+
+        final ComponentIdGenerator componentIdGenerator = this.syncOptions.getComponentIdGenerator();
+        final String parameterContextId = componentIdGenerator.generateUuid(versionedParameterContext.getName(), versionedParameterContext.getName(), versionedParameterContext.getName());
+
+        final Map<String, Parameter> parameters = new HashMap<>();
+        for (final VersionedParameter versionedParameter : versionedParameterContext.getParameters()) {
+            final ParameterDescriptor descriptor = new ParameterDescriptor.Builder()
+                .name(versionedParameter.getName())
+                .description(versionedParameter.getDescription())
+                .sensitive(versionedParameter.isSensitive())
+                .build();
+
+            final Parameter parameter = new Parameter(descriptor, versionedParameter.getValue());
+            parameters.put(versionedParameter.getName(), parameter);
+        }
+
+        return context.getFlowManager().createParameterContext(parameterContextId, versionedParameterContext.getName(), parameters, Collections.emptyList());
+    }
+
+    private ParameterContext createParameterContext(final VersionedParameterContext versionedParameterContext, final String parameterContextId,
+                                                    final Map<String, VersionedParameterContext> versionedParameterContexts) {
+        final Map<String, Parameter> parameters = new HashMap<>();
+        for (final VersionedParameter versionedParameter : versionedParameterContext.getParameters()) {
+            final ParameterDescriptor descriptor = new ParameterDescriptor.Builder()
+                .name(versionedParameter.getName())
+                .description(versionedParameter.getDescription())
+                .sensitive(versionedParameter.isSensitive())
+                .build();
+
+            final Parameter parameter = new Parameter(descriptor, versionedParameter.getValue());
+            parameters.put(versionedParameter.getName(), parameter);
+        }
+
+        final List<String> parameterContextRefs = new ArrayList<>();
+        if (versionedParameterContext.getInheritedParameterContexts() != null) {
+            versionedParameterContext.getInheritedParameterContexts().stream()
+                .map(name -> createParameterReferenceId(name, versionedParameterContexts))
+                .forEach(parameterContextRefs::add);
+        }
+
+        return context.getFlowManager().createParameterContext(parameterContextId, versionedParameterContext.getName(), parameters, parameterContextRefs);
+    }
+
+    private String createParameterReferenceId(final String parameterContextName, final Map<String, VersionedParameterContext> versionedParameterContexts) {
+        final VersionedParameterContext versionedParameterContext = versionedParameterContexts.get(parameterContextName);
+        final ParameterContext selectedParameterContext = selectParameterContext(versionedParameterContext, versionedParameterContexts);
+        return selectedParameterContext.getIdentifier();
+    }
+
+    private ParameterContext selectParameterContext(final VersionedParameterContext versionedParameterContext, final Map<String, VersionedParameterContext> versionedParameterContexts) {
+        final ParameterContext contextByName = getParameterContextByName(versionedParameterContext.getName());
+        final ParameterContext selectedParameterContext;
+        if (contextByName == null) {
+            final String parameterContextId = context.getFlowMappingOptions().getComponentIdLookup().getComponentId(Optional.ofNullable(versionedParameterContext.getIdentifier()),
+                versionedParameterContext.getInstanceIdentifier());
+            selectedParameterContext = createParameterContext(versionedParameterContext, parameterContextId, versionedParameterContexts);
+        } else {
+            selectedParameterContext = contextByName;
+            addMissingConfiguration(versionedParameterContext, selectedParameterContext, versionedParameterContexts);
+        }
+
+        return selectedParameterContext;
+    }
+
+
+    private void addMissingConfiguration(final VersionedParameterContext versionedParameterContext, final ParameterContext currentParameterContext,
+                                         final Map<String, VersionedParameterContext> versionedParameterContexts) {
+        final Map<String, Parameter> parameters = new HashMap<>();
+        for (final VersionedParameter versionedParameter : versionedParameterContext.getParameters()) {
+            final Optional<Parameter> parameterOption = currentParameterContext.getParameter(versionedParameter.getName());
+            if (parameterOption.isPresent()) {
+                // Skip this parameter, since it is already defined. We only want to add missing parameters
+                continue;
+            }
+
+            final ParameterDescriptor descriptor = new ParameterDescriptor.Builder()
+                .name(versionedParameter.getName())
+                .description(versionedParameter.getDescription())
+                .sensitive(versionedParameter.isSensitive())
+                .build();
+
+            final Parameter parameter = new Parameter(descriptor, versionedParameter.getValue());
+            parameters.put(versionedParameter.getName(), parameter);
+        }
+
+        currentParameterContext.setParameters(parameters);
+
+
+        // If the current parameter context doesn't have any inherited param contexts but the versioned one does,
+        // add the versioned ones.
+        if (versionedParameterContext.getInheritedParameterContexts() != null && !versionedParameterContext.getInheritedParameterContexts().isEmpty()
+            && currentParameterContext.getInheritedParameterContexts().isEmpty()) {
+            currentParameterContext.setInheritedParameterContexts(versionedParameterContext.getInheritedParameterContexts().stream()
+                .map(name -> selectParameterContext(versionedParameterContexts.get(name), versionedParameterContexts))
+                .collect(Collectors.toList()));
+        }
+    }
+
+    private boolean isEqual(final BundleCoordinate coordinate, final Bundle bundle) {
+        if (!bundle.getGroup().equals(coordinate.getGroup())) {
+            return false;
+        }
+
+        if (!bundle.getArtifact().equals(coordinate.getId())) {
+            return false;
+        }
+
+        return bundle.getVersion().equals(coordinate.getVersion());
+    }
+
+    private BundleCoordinate toCoordinate(final Bundle bundle) {
+        return new BundleCoordinate(bundle.getGroup(), bundle.getArtifact(), bundle.getVersion());
+    }
+
+    private Map<String, VersionedParameterContext> getVersionedParameterContexts(final VersionedFlowCoordinates versionedFlowCoordinates) {
+        final String registryId = context.getFlowRegistryClient().getFlowRegistryId(versionedFlowCoordinates.getRegistryUrl());
+        if (registryId == null) {
+            throw new ResourceNotFoundException("Could not find any Flow Registry registered with url: " + versionedFlowCoordinates.getRegistryUrl());
+        }
+
+        final FlowRegistry flowRegistry = context.getFlowRegistryClient().getFlowRegistry(registryId);
+        if (flowRegistry == null) {
+            throw new ResourceNotFoundException("Could not find any Flow Registry registered with identifier " + registryId);
+        }
+
+        final String bucketId = versionedFlowCoordinates.getBucketId();
+        final String flowId = versionedFlowCoordinates.getFlowId();
+        final int flowVersion = versionedFlowCoordinates.getVersion();
+
+        try {
+            final VersionedFlowSnapshot childSnapshot = flowRegistry.getFlowContents(bucketId, flowId, flowVersion, false);
+            return childSnapshot.getParameterContexts();
+        } catch (final NiFiRegistryException e) {
+            throw new IllegalArgumentException("The Flow Registry with ID " + registryId + " reports that no Flow exists with Bucket "
+                + bucketId + ", Flow " + flowId + ", Version " + flowVersion, e);
+        } catch (final IOException ioe) {
+            throw new IllegalStateException("Failed to communicate with Flow Registry when attempting to retrieve a versioned flow");
+        }
+    }
+
+    private void updateFunnel(final Funnel funnel, final VersionedFunnel proposed) {
+        funnel.setPosition(new Position(proposed.getPosition().getX(), proposed.getPosition().getY()));
+    }
+
+    private Funnel addFunnel(final ProcessGroup destination, final VersionedFunnel proposed, final ComponentIdGenerator componentIdGenerator) {
+        final String id = componentIdGenerator.generateUuid(proposed.getIdentifier(), proposed.getInstanceIdentifier(), destination.getIdentifier());
+        final Funnel funnel = context.getFlowManager().createFunnel(id);
+        funnel.setVersionedComponentId(proposed.getIdentifier());
+        destination.addFunnel(funnel);
+        updateFunnel(funnel, proposed);
+
+        return funnel;
+    }
+
+    private boolean isUpdateable(final Connection connection) {
+        final Optional<String> versionIdOptional = connection.getVersionedComponentId();
+        if (versionIdOptional.isPresent() && !updatedVersionedComponentIds.contains(versionIdOptional.get())) {
+            return false;
+        }
+
+        final Connectable source = connection.getSource();
+        if (source.getConnectableType() != ConnectableType.FUNNEL && source.isRunning()) {
+            return false;
+        }
+
+        final Connectable destination = connection.getDestination();
+        return destination.getConnectableType() == ConnectableType.FUNNEL || !destination.isRunning();
+    }
+
+    private String generateTemporaryPortName(final VersionedPort proposedPort) {
+        final String versionedPortId = proposedPort.getIdentifier();
+        final String proposedPortFinalName = proposedPort.getName();
+        return proposedPortFinalName + " (" + versionedPortId + ")";
+    }
+
+    private void updatePortToSetFinalName(final Port port, final String name) {
+        port.setName(name);
+    }
+
+    private void updatePort(final Port port, final VersionedPort proposed, final String temporaryName) {
+        final String name = temporaryName != null ? temporaryName : proposed.getName();
+        port.setComments(proposed.getComments());
+        port.setName(name);
+        port.setPosition(new Position(proposed.getPosition().getX(), proposed.getPosition().getY()));
+        port.setMaxConcurrentTasks(proposed.getConcurrentlySchedulableTaskCount());
+
+        final org.apache.nifi.flow.ScheduledState scheduledState = proposed.getScheduledState() == null ? org.apache.nifi.flow.ScheduledState.ENABLED : proposed.getScheduledState();
+
+        final ProcessGroup group = port.getProcessGroup();
+        if (port.getConnectableType() == ConnectableType.INPUT_PORT) {
+            switch (scheduledState) {
+                case DISABLED:
+                    group.disableInputPort(port);
+                    break;
+                case ENABLED:
+                    if (port.getScheduledState() == ScheduledState.DISABLED) {
+                        group.enableInputPort(port);
+                    } else if (port.getScheduledState() == ScheduledState.RUNNING) {
+                        group.stopInputPort(port);
+                    }
+                    break;
+                case RUNNING:
+                    if (port.getScheduledState() == ScheduledState.DISABLED) {
+                        group.enableInputPort(port);
+                    }
+                    if (port.getScheduledState() == ScheduledState.STOPPED) {
+                        context.getComponentScheduler().startComponent(port);
+                    }
+                    break;
+            }
+        } else if (port.getConnectableType() == ConnectableType.OUTPUT_PORT) {
+            switch (scheduledState) {
+                case DISABLED:
+                    group.disableOutputPort(port);
+                    break;
+                case ENABLED:
+                    if (port.getScheduledState() == ScheduledState.DISABLED) {
+                        group.enableOutputPort(port);
+                    } else if (port.getScheduledState() == ScheduledState.RUNNING) {
+                        group.stopOutputPort(port);
+                    }
+                    break;
+                case RUNNING:
+                    if (port.getScheduledState() == ScheduledState.DISABLED) {
+                        group.enableOutputPort(port);
+                    }
+                    if (port.getScheduledState() == ScheduledState.STOPPED) {
+                        context.getComponentScheduler().startComponent(port);
+                    }
+                    break;
+            }
+        }
+    }
+
+    private Port addInputPort(final ProcessGroup destination, final VersionedPort proposed, final ComponentIdGenerator componentIdGenerator, final String temporaryName) {
+        final String name = temporaryName != null ? temporaryName : proposed.getName();
+
+        final String id = componentIdGenerator.generateUuid(proposed.getIdentifier(), proposed.getInstanceIdentifier(), destination.getIdentifier());
+
+        final Port port;
+        if (proposed.isAllowRemoteAccess()) {
+            port = context.getFlowManager().createPublicInputPort(id, name);
+        } else {
+            port = context.getFlowManager().createLocalInputPort(id, name);
+        }
+
+        port.setVersionedComponentId(proposed.getIdentifier());
+        destination.addInputPort(port);
+        updatePort(port, proposed, temporaryName);
+
+        return port;
+    }
+
+    private Port addOutputPort(final ProcessGroup destination, final VersionedPort proposed, final ComponentIdGenerator componentIdGenerator, final String temporaryName) {
+        final String name = temporaryName != null ? temporaryName : proposed.getName();
+        final String id = componentIdGenerator.generateUuid(proposed.getIdentifier(), proposed.getInstanceIdentifier(), destination.getIdentifier());
+
+        final Port port;
+        if (proposed.isAllowRemoteAccess()) {
+            port = context.getFlowManager().createPublicOutputPort(id, name);
+        } else {
+            port = context.getFlowManager().createLocalOutputPort(id, name);
+        }
+
+        port.setVersionedComponentId(proposed.getIdentifier());
+        destination.addOutputPort(port);
+        updatePort(port, proposed, temporaryName);
+
+        return port;
+    }
+
+    private Label addLabel(final ProcessGroup destination, final VersionedLabel proposed, final ComponentIdGenerator componentIdGenerator) {
+        final String id = componentIdGenerator.generateUuid(proposed.getIdentifier(), proposed.getInstanceIdentifier(), destination.getIdentifier());
+        final Label label = context.getFlowManager().createLabel(id, proposed.getLabel());
+        label.setVersionedComponentId(proposed.getIdentifier());
+        destination.addLabel(label);
+        updateLabel(label, proposed);
+
+        return label;
+    }
+
+    private void updateLabel(final Label label, final VersionedLabel proposed) {
+        label.setPosition(new Position(proposed.getPosition().getX(), proposed.getPosition().getY()));
+        label.setSize(new Size(proposed.getWidth(), proposed.getHeight()));
+        label.setStyle(proposed.getStyle());
+        label.setValue(proposed.getLabel());
+    }
+
+    private ProcessorNode addProcessor(final ProcessGroup destination, final VersionedProcessor proposed, final ComponentIdGenerator componentIdGenerator) throws ProcessorInstantiationException {
+        final String identifier = componentIdGenerator.generateUuid(proposed.getIdentifier(), proposed.getInstanceIdentifier(), destination.getIdentifier());
+        LOG.debug("Adding Processor with ID {} of type {}", identifier, proposed.getType());
+
+        final BundleCoordinate coordinate = toCoordinate(proposed.getBundle());
+        final ProcessorNode procNode = context.getFlowManager().createProcessor(proposed.getType(), identifier, coordinate, true);
+        procNode.setVersionedComponentId(proposed.getIdentifier());
+
+        destination.addProcessor(procNode);
+        updateProcessor(procNode, proposed);
+
+        // Notify the processor node that the configuration (properties, e.g.) has been restored
+        final ProcessContext processContext = context.getProcessContextFactory().apply(procNode);
+        procNode.onConfigurationRestored(processContext);
+
+        return procNode;
+    }
+
+    private void updateProcessor(final ProcessorNode processor, final VersionedProcessor proposed) throws ProcessorInstantiationException {
+        LOG.debug("Updating Processor {}", processor);
+
+        processor.pauseValidationTrigger();
+        try {
+            processor.setAnnotationData(proposed.getAnnotationData());
+            processor.setBulletinLevel(LogLevel.valueOf(proposed.getBulletinLevel()));
+            processor.setComments(proposed.getComments());
+            processor.setName(proposed.getName());
+            processor.setPenalizationPeriod(proposed.getPenaltyDuration());
+
+            final Map<String, String> properties = populatePropertiesMap(processor, proposed.getProperties(), proposed.getPropertyDescriptors(), processor.getProcessGroup());
+            processor.setProperties(properties, true);
+            processor.setRunDuration(proposed.getRunDurationMillis(), TimeUnit.MILLISECONDS);
+            processor.setSchedulingStrategy(SchedulingStrategy.valueOf(proposed.getSchedulingStrategy()));
+            processor.setScheduldingPeriod(proposed.getSchedulingPeriod());
+            processor.setMaxConcurrentTasks(proposed.getConcurrentlySchedulableTaskCount());
+            processor.setExecutionNode(ExecutionNode.valueOf(proposed.getExecutionNode()));
+            processor.setStyle(proposed.getStyle());
+            processor.setYieldPeriod(proposed.getYieldDuration());
+            processor.setPosition(new Position(proposed.getPosition().getX(), proposed.getPosition().getY()));
+
+            final ScheduledState procState = processor.getScheduledState();
+            final ProcessGroup group = processor.getProcessGroup();
+            switch (proposed.getScheduledState()) {
+                case DISABLED:
+                    if (procState == ScheduledState.RUNNING) {
+                        LOG.debug("Stopping {}", processor);
+                        group.stopProcessor(processor);
+                    }
+
+                    LOG.debug("Disabling {}", processor);
+                    group.disableProcessor(processor);
+                    break;
+                case ENABLED:
+                    if (procState == ScheduledState.DISABLED) {
+                        LOG.debug("Enabling {}", processor);
+                        group.enableProcessor(processor);
+                    } else if (procState == ScheduledState.RUNNING) {
+                        LOG.debug("Stopping {}", processor);
+                        group.stopProcessor(processor);
+                    }
+                    break;
+                case RUNNING:
+                    if (procState == ScheduledState.DISABLED) {
+                        LOG.debug("Enabling {}", processor);
+                        group.enableProcessor(processor);
+                    }
+                    LOG.debug("Starting {}", processor);
+                    context.getComponentScheduler().startComponent(processor);
+                    break;
+            }
+
+            if (!isEqual(processor.getBundleCoordinate(), proposed.getBundle())) {
+                final BundleCoordinate newBundleCoordinate = toCoordinate(proposed.getBundle());
+                final List<PropertyDescriptor> descriptors = new ArrayList<>(processor.getProperties().keySet());
+                final Set<URL> additionalUrls = processor.getAdditionalClasspathResources(descriptors);
+                context.getReloadComponent().reload(processor, proposed.getType(), newBundleCoordinate, additionalUrls);
+            }
+        } finally {
+            processor.resumeValidationTrigger();
+        }
+    }
+
+
+    private String getServiceInstanceId(final String serviceVersionedComponentId, final ProcessGroup group) {
+        for (final ControllerServiceNode serviceNode : group.getControllerServices(false)) {
+            final String versionedId = serviceNode.getVersionedComponentId().orElse(
+                NiFiRegistryFlowMapper.generateVersionedComponentId(serviceNode.getIdentifier()));
+            if (versionedId.equals(serviceVersionedComponentId)) {
+                return serviceNode.getIdentifier();
+            }
+        }
+
+        final ProcessGroup parent = group.getParent();
+        if (parent == null) {
+            return null;
+        }
+
+        return getServiceInstanceId(serviceVersionedComponentId, parent);
+
+    }
+
+    private RemoteProcessGroup addRemoteProcessGroup(final ProcessGroup destination, final VersionedRemoteProcessGroup proposed, final ComponentIdGenerator componentIdGenerator) {
+        final String id = componentIdGenerator.generateUuid(proposed.getIdentifier(), proposed.getInstanceIdentifier(), destination.getIdentifier());
+        final RemoteProcessGroup rpg = context.getFlowManager().createRemoteProcessGroup(id, proposed.getTargetUris());
+        rpg.setVersionedComponentId(proposed.getIdentifier());
+
+        destination.addRemoteProcessGroup(rpg);
+        updateRemoteProcessGroup(rpg, proposed, componentIdGenerator);
+
+        return rpg;
+    }
+
+    private void updateRemoteProcessGroup(final RemoteProcessGroup rpg, final VersionedRemoteProcessGroup proposed, final ComponentIdGenerator componentIdGenerator) {
+        rpg.setComments(proposed.getComments());
+        rpg.setCommunicationsTimeout(proposed.getCommunicationsTimeout());
+        rpg.setInputPorts(proposed.getInputPorts() == null ? Collections.emptySet() : proposed.getInputPorts().stream()
+            .map(port -> createPortDescriptor(port, componentIdGenerator, rpg.getIdentifier()))
+            .collect(Collectors.toSet()), false);
+        rpg.setName(proposed.getName());
+        rpg.setNetworkInterface(proposed.getLocalNetworkInterface());
+        rpg.setOutputPorts(proposed.getOutputPorts() == null ? Collections.emptySet() : proposed.getOutputPorts().stream()
+            .map(port -> createPortDescriptor(port, componentIdGenerator, rpg.getIdentifier()))
+            .collect(Collectors.toSet()), false);
+        rpg.setPosition(new Position(proposed.getPosition().getX(), proposed.getPosition().getY()));
+        rpg.setProxyHost(proposed.getProxyHost());
+        rpg.setProxyPort(proposed.getProxyPort());
+        rpg.setProxyUser(proposed.getProxyUser());
+        rpg.setTransportProtocol(SiteToSiteTransportProtocol.valueOf(proposed.getTransportProtocol()));
+        rpg.setYieldDuration(proposed.getYieldDuration());
+
+        if (syncOptions.isUpdateRpgUrls()) {
+            rpg.setTargetUris(proposed.getTargetUris());
+        }
+
+        if (proposed.getInputPorts() != null) {
+            for (final VersionedRemoteGroupPort port : proposed.getInputPorts()) {
+                if (port.getScheduledState() != org.apache.nifi.flow.ScheduledState.RUNNING) {
+                    continue;
+                }
+
+                final String portId = componentIdGenerator.generateUuid(proposed.getIdentifier(), proposed.getInstanceIdentifier(), rpg.getIdentifier());
+                final RemoteGroupPort remoteGroupPort = rpg.getInputPort(portId);
+                if (remoteGroupPort != null) {
+                    context.getComponentScheduler().startComponent(remoteGroupPort);
+                }
+            }
+        }
+
+        if (proposed.getOutputPorts() != null) {
+            for (final VersionedRemoteGroupPort port : proposed.getOutputPorts()) {
+                if (port.getScheduledState() != org.apache.nifi.flow.ScheduledState.RUNNING) {
+                    continue;
+                }
+
+                final String portId = componentIdGenerator.generateUuid(proposed.getIdentifier(), proposed.getInstanceIdentifier(), rpg.getIdentifier());
+                final RemoteGroupPort remoteGroupPort = rpg.getOutputPort(portId);
+                if (remoteGroupPort != null) {
+                    context.getComponentScheduler().startComponent(remoteGroupPort);
+                }
+            }
+        }
+    }
+
+
+    private RemoteProcessGroupPortDescriptor createPortDescriptor(final VersionedRemoteGroupPort proposed, final ComponentIdGenerator componentIdGenerator, final String rpgId) {
+        final StandardRemoteProcessGroupPortDescriptor descriptor = new StandardRemoteProcessGroupPortDescriptor();
+        descriptor.setVersionedComponentId(proposed.getIdentifier());
+
+        final BatchSize batchSize = proposed.getBatchSize();
+        if (batchSize != null) {
+            descriptor.setBatchCount(batchSize.getCount());
+            descriptor.setBatchDuration(batchSize.getDuration());
+            descriptor.setBatchSize(batchSize.getSize());
+        }
+
+        descriptor.setComments(proposed.getComments());
+        descriptor.setConcurrentlySchedulableTaskCount(proposed.getConcurrentlySchedulableTaskCount());
+        descriptor.setGroupId(proposed.getRemoteGroupId());
+        descriptor.setTargetId(proposed.getTargetId());
+
+        final String id = componentIdGenerator.generateUuid(proposed.getIdentifier(), proposed.getInstanceIdentifier(), rpgId);
+        descriptor.setId(id);
+        descriptor.setName(proposed.getName());
+        descriptor.setUseCompression(proposed.isUseCompression());
+        return descriptor;
+    }
+
+
+    private void updateConnection(final Connection connection, final VersionedConnection proposed) {
+        LOG.debug("Updating connection from {} to {} with name {} and relationships {}: {}",
+            proposed.getSource(), proposed.getDestination(), proposed.getName(), proposed.getSelectedRelationships(), connection);
+
+        connection.setBendPoints(proposed.getBends() == null ? Collections.emptyList() :
+            proposed.getBends().stream()
+                .map(pos -> new Position(pos.getX(), pos.getY()))
+                .collect(Collectors.toList()));
+
+        connection.setDestination(getConnectable(connection.getProcessGroup(), proposed.getDestination()));
+        connection.setLabelIndex(proposed.getLabelIndex());
+        connection.setName(proposed.getName());
+        connection.setRelationships(proposed.getSelectedRelationships().stream()
+            .map(name -> new Relationship.Builder().name(name).build())
+            .collect(Collectors.toSet()));
+        connection.setZIndex(proposed.getzIndex());
+
+        final FlowFileQueue queue = connection.getFlowFileQueue();
+        queue.setBackPressureDataSizeThreshold(proposed.getBackPressureDataSizeThreshold());
+        queue.setBackPressureObjectThreshold(proposed.getBackPressureObjectThreshold());
+        queue.setFlowFileExpiration(proposed.getFlowFileExpiration());
+
+        final List<FlowFilePrioritizer> prioritizers = proposed.getPrioritizers() == null ? Collections.emptyList() : proposed.getPrioritizers().stream()
+            .map(prioritizerName -> {
+                try {
+                    return context.getFlowManager().createPrioritizer(prioritizerName);
+                } catch (final Exception e) {
+                    throw new IllegalStateException("Failed to create Prioritizer of type " + prioritizerName + " for Connection with ID " + connection.getIdentifier());
+                }
+            })
+            .collect(Collectors.toList());
+
+        queue.setPriorities(prioritizers);
+
+        final String loadBalanceStrategyName = proposed.getLoadBalanceStrategy();
+        if (loadBalanceStrategyName == null) {
+            queue.setLoadBalanceStrategy(LoadBalanceStrategy.DO_NOT_LOAD_BALANCE, proposed.getPartitioningAttribute());
+        } else {
+            final LoadBalanceStrategy loadBalanceStrategy = LoadBalanceStrategy.valueOf(loadBalanceStrategyName);
+            final String partitioningAttribute = proposed.getPartitioningAttribute();
+
+            queue.setLoadBalanceStrategy(loadBalanceStrategy, partitioningAttribute);
+        }
+
+        final String compressionName = proposed.getLoadBalanceCompression();
+        if (compressionName == null) {
+            queue.setLoadBalanceCompression(LoadBalanceCompression.DO_NOT_COMPRESS);
+        } else {
+            queue.setLoadBalanceCompression(LoadBalanceCompression.valueOf(compressionName));
+        }
+    }
+
+    private Connection addConnection(final ProcessGroup destinationGroup, final VersionedConnection proposed, final ComponentIdGenerator componentIdGenerator) {
+        LOG.debug("Adding connection from {} to {} with name {} and relationships {}", proposed.getSource(), proposed.getDestination(), proposed.getName(), proposed.getSelectedRelationships());
+
+        final Connectable source = getConnectable(destinationGroup, proposed.getSource());
+        if (source == null) {
+            throw new IllegalArgumentException("Connection has a source with identifier " + proposed.getIdentifier()
+                + " but no component could be found in the Process Group with a corresponding identifier");
+        }
+
+        final Connectable destination = getConnectable(destinationGroup, proposed.getDestination());
+        if (destination == null) {
+            throw new IllegalArgumentException("Connection has a destination with identifier " + proposed.getDestination().getId()
+                + " but no component could be found in the Process Group with a corresponding identifier");
+        }
+
+        final String id = componentIdGenerator.generateUuid(proposed.getIdentifier(), proposed.getInstanceIdentifier(), destination.getIdentifier());
+        final Connection connection = context.getFlowManager().createConnection(id, proposed.getName(), source, destination, proposed.getSelectedRelationships());
+        connection.setVersionedComponentId(proposed.getIdentifier());
+        destinationGroup.addConnection(connection);
+        updateConnection(connection, proposed);
+
+        context.getFlowManager().onConnectionAdded(connection);
+        return connection;
+    }
+
+    private Connectable getConnectable(final ProcessGroup group, final ConnectableComponent connectableComponent) {
+        final String id = connectableComponent.getId();
+
+        switch (connectableComponent.getType()) {
+            case FUNNEL:
+                return group.getFunnels().stream()
+                    .filter(component -> matchesId(component, id))
+                    .findAny()
+                    .orElse(null);
+            case INPUT_PORT: {
+                final Optional<Port> port = group.getInputPorts().stream()
+                    .filter(component -> matchesId(component, id))
+                    .findAny();
+
+                if (port.isPresent()) {
+                    return port.get();
+                }
+
+                // Attempt to locate child group by versioned component id
+                final Optional<ProcessGroup> optionalSpecifiedGroup = group.getProcessGroups().stream()
+                    .filter(child -> matchesGroupId(child, connectableComponent.getGroupId()))
+                    .findFirst();
+
+                if (optionalSpecifiedGroup.isPresent()) {
+                    final ProcessGroup specifiedGroup = optionalSpecifiedGroup.get();
+                    return specifiedGroup.getInputPorts().stream()
+                        .filter(component -> matchesId(component, id))
+                        .findAny()
+                        .orElse(null);
+                }
+
+                // If no child group matched the versioned component id, then look at all child groups. This is done because
+                // in older versions, we did not properly map Versioned Component ID's to Ports' parent groups. As a result,
+                // if the flow doesn't contain the properly mapped group id, we need to search all child groups.
+                return group.getProcessGroups().stream()
+                    .flatMap(gr -> gr.getInputPorts().stream())
+                    .filter(component -> matchesId(component, id))
+                    .findAny()
+                    .orElse(null);
+            }
+            case OUTPUT_PORT: {
+                final Optional<Port> port = group.getOutputPorts().stream()
+                    .filter(component -> matchesId(component, id))
+                    .findAny();
+
+                if (port.isPresent()) {
+                    return port.get();
+                }
+
+                // Attempt to locate child group by versioned component id
+                final Optional<ProcessGroup> optionalSpecifiedGroup = group.getProcessGroups().stream()
+                    .filter(child -> matchesGroupId(child, connectableComponent.getGroupId()))
+                    .findFirst();
+
+                if (optionalSpecifiedGroup.isPresent()) {
+                    final ProcessGroup specifiedGroup = optionalSpecifiedGroup.get();
+                    return specifiedGroup.getOutputPorts().stream()
+                        .filter(component -> matchesId(component, id))
+                        .findAny()
+                        .orElse(null);
+                }
+
+                // If no child group matched the versioned component id, then look at all child groups. This is done because
+                // in older versions, we did not properly map Versioned Component ID's to Ports' parent groups. As a result,
+                // if the flow doesn't contain the properly mapped group id, we need to search all child groups.
+                return group.getProcessGroups().stream()
+                    .flatMap(gr -> gr.getOutputPorts().stream())
+                    .filter(component -> matchesId(component, id))
+                    .findAny()
+                    .orElse(null);
+            }
+            case PROCESSOR:
+                return group.getProcessors().stream()
+                    .filter(component -> matchesId(component, id))
+                    .findAny()
+                    .orElse(null);
+            case REMOTE_INPUT_PORT: {
+                final String rpgId = connectableComponent.getGroupId();
+                final Optional<RemoteProcessGroup> rpgOption = group.getRemoteProcessGroups().stream()
+                    .filter(component -> rpgId.equals(component.getIdentifier()) || rpgId.equals(component.getVersionedComponentId().orElse(
+                        NiFiRegistryFlowMapper.generateVersionedComponentId(component.getIdentifier()))))
+                    .findAny();
+
+                if (!rpgOption.isPresent()) {
+                    throw new IllegalArgumentException("Connection refers to a Port with ID " + id + " within Remote Process Group with ID "
+                        + rpgId + " but could not find a Remote Process Group corresponding to that ID");
+                }
+
+                final RemoteProcessGroup rpg = rpgOption.get();
+                final Optional<RemoteGroupPort> portByIdOption = rpg.getInputPorts().stream()
+                    .filter(component -> matchesId(component, id))
+                    .findAny();
+
+                if (portByIdOption.isPresent()) {
+                    return portByIdOption.get();
+                }
+
+                return rpg.getInputPorts().stream()
+                    .filter(component -> connectableComponent.getName().equals(component.getName()))
+                    .findAny()
+                    .orElse(null);
+            }
+            case REMOTE_OUTPUT_PORT: {
+                final String rpgId = connectableComponent.getGroupId();
+                final Optional<RemoteProcessGroup> rpgOption = group.getRemoteProcessGroups().stream()
+                    .filter(component -> rpgId.equals(component.getIdentifier()) || rpgId.equals(component.getVersionedComponentId().orElse(
+                        NiFiRegistryFlowMapper.generateVersionedComponentId(component.getIdentifier()))))
+                    .findAny();
+
+                if (!rpgOption.isPresent()) {
+                    throw new IllegalArgumentException("Connection refers to a Port with ID " + id + " within Remote Process Group with ID "
+                        + rpgId + " but could not find a Remote Process Group corresponding to that ID");
+                }
+
+                final RemoteProcessGroup rpg = rpgOption.get();
+                final Optional<RemoteGroupPort> portByIdOption = rpg.getOutputPorts().stream()
+                    .filter(component -> matchesId(component, id))
+                    .findAny();
+
+                if (portByIdOption.isPresent()) {
+                    return portByIdOption.get();
+                }
+
+                return rpg.getOutputPorts().stream()
+                    .filter(component -> connectableComponent.getName().equals(component.getName()))
+                    .findAny()
+                    .orElse(null);
+            }
+        }
+
+        return null;
+    }
+
+    private <T extends org.apache.nifi.components.VersionedComponent & Connectable> boolean matchesId(final T component, final String id) {
+        return id.equals(component.getIdentifier()) || id.equals(component.getVersionedComponentId().orElse(NiFiRegistryFlowMapper.generateVersionedComponentId(component.getIdentifier())));
+    }
+
+    private boolean matchesGroupId(final ProcessGroup group, final String groupId) {
+        return groupId.equals(group.getIdentifier()) || group.getVersionedComponentId().orElse(
+            NiFiRegistryFlowMapper.generateVersionedComponentId(group.getIdentifier())).equals(groupId);
+    }
+
+
+    private void findAllProcessors(final VersionedProcessGroup group, final Map<String, VersionedProcessor> map) {
+        for (final VersionedProcessor processor : group.getProcessors()) {
+            map.put(processor.getIdentifier(), processor);
+        }
+
+        for (final VersionedProcessGroup childGroup : group.getProcessGroups()) {
+            findAllProcessors(childGroup, map);
+        }
+    }
+
+    private void findAllControllerServices(final VersionedProcessGroup group, final Map<String, VersionedControllerService> map) {
+        for (final VersionedControllerService service : group.getControllerServices()) {
+            map.put(service.getIdentifier(), service);
+        }
+
+        for (final VersionedProcessGroup childGroup : group.getProcessGroups()) {
+            findAllControllerServices(childGroup, map);
+        }
+    }
+
+    private void findAllConnections(final VersionedProcessGroup group, final Map<String, VersionedConnection> map) {
+        for (final VersionedConnection connection : group.getConnections()) {
+            map.put(connection.getIdentifier(), connection);
+        }
+
+        for (final VersionedProcessGroup childGroup : group.getProcessGroups()) {
+            findAllConnections(childGroup, map);
+        }
+    }
+
+    /**
+     * Match components of the given process group to the proposed versioned process group and verify missing components
+     * are in a state that they can be safely removed. Specifically, check for removed child process groups and descendants.
+     * Disallow removal of groups with attached templates. Optionally also check for removed connections with data in their
+     * queue, either because the connections were removed from a matched process group or their group itself was removed.
+     *
+     * @param processGroup the current process group to examine
+     * @param proposedGroup the proposed versioned process group to match with
+     * @param verifyConnectionRemoval whether or not to verify that connections that are not present in the proposed flow can be removed
+     */
+    private void verifyCanRemoveMissingComponents(final ProcessGroup processGroup, final VersionedProcessGroup proposedGroup,
+                                                  final boolean verifyConnectionRemoval) {
+        if (verifyConnectionRemoval) {
+            final Map<String, VersionedConnection> proposedConnectionsByVersionedId = proposedGroup.getConnections().stream()
+                .collect(Collectors.toMap(VersionedComponent::getIdentifier, Function.identity()));
+
+            // match group's current connections to proposed connections to determine if they've been removed
+            for (final Connection connection : processGroup.getConnections()) {
+                final String versionedId = connection.getVersionedComponentId().orElse(
+                    NiFiRegistryFlowMapper.generateVersionedComponentId(connection.getIdentifier()));
+                final VersionedConnection proposedConnection = proposedConnectionsByVersionedId.get(versionedId);
+                if (proposedConnection == null) {
+                    // connection doesn't exist in proposed connections, make sure it doesn't have any data in it
+                    final FlowFileQueue flowFileQueue = connection.getFlowFileQueue();
+                    if (!flowFileQueue.isEmpty()) {
+                        throw new IllegalStateException(processGroup + " cannot be updated to the proposed flow because the proposed flow "
+                            + "does not contain a match for " + connection + " and the connection currently has data in the queue.");
+                    }
+                }
+            }
+        }
+
+        final Map<String, VersionedProcessGroup> proposedGroupsByVersionedId = proposedGroup.getProcessGroups().stream()
+            .collect(Collectors.toMap(VersionedComponent::getIdentifier, Function.identity()));
+
+        // match current child groups to proposed child groups to determine if they've been removed
+        for (final ProcessGroup childGroup : processGroup.getProcessGroups()) {
+            final String versionedId = childGroup.getVersionedComponentId().orElse(
+                NiFiRegistryFlowMapper.generateVersionedComponentId(childGroup.getIdentifier()));
+            final VersionedProcessGroup proposedChildGroup = proposedGroupsByVersionedId.get(versionedId);
+            if (proposedChildGroup == null) {
+                // child group will be removed, check group and descendants for attached templates
+                final Template removedTemplate = childGroup.findAllTemplates().stream().findFirst().orElse(null);
+                if (removedTemplate != null) {
+                    throw new IllegalStateException(processGroup + " cannot be updated to the proposed flow because the child " + removedTemplate.getProcessGroup()
+                        + " that exists locally has one or more Templates, and the proposed flow does not contain these templates. "
+                        + "A Process Group cannot be deleted while it contains Templates. Please remove the Templates before re-attempting.");
+                }
+                if (verifyConnectionRemoval) {
+                    // check removed group and its descendants for connections with data in the queue
+                    final Connection removedConnection = childGroup.findAllConnections().stream()
+                        .filter(connection -> !connection.getFlowFileQueue().isEmpty()).findFirst().orElse(null);
+                    if (removedConnection != null) {
+                        throw new IllegalStateException(processGroup + " cannot be updated to the proposed flow because the proposed flow "
+                            + "does not contain a match for " + removedConnection + " and the connection currently has data in the queue.");
+                    }
+                }
+            } else {
+                // child group successfully matched, recurse into verification of its contents
+                verifyCanRemoveMissingComponents(childGroup, proposedChildGroup, verifyConnectionRemoval);
+            }
+        }
+    }
+
+
+    private Set<String> getKnownVariableNames(final ProcessGroup group) {
+        final Set<String> variableNames = new HashSet<>();
+        populateKnownVariableNames(group, variableNames);
+        return variableNames;
+    }
+
+    private void populateKnownVariableNames(final ProcessGroup group, final Set<String> knownVariables) {
+        group.getVariableRegistry().getVariableMap().keySet().stream()
+            .map(VariableDescriptor::getName)
+            .forEach(knownVariables::add);
+
+        final ProcessGroup parent = group.getParent();
+        if (parent != null) {
+            populateKnownVariableNames(parent, knownVariables);
+        }
+    }
+
+    private ControllerServiceNode getVersionedControllerService(final ProcessGroup group, final String versionedComponentId) {
+        if (group == null) {
+            return null;
+        }
+
+        for (final ControllerServiceNode serviceNode : group.getControllerServices(false)) {
+            final String serviceNodeVersionedComponentId = serviceNode.getVersionedComponentId().orElse(
+                NiFiRegistryFlowMapper.generateVersionedComponentId(serviceNode.getIdentifier()));
+            if (serviceNodeVersionedComponentId.equals(versionedComponentId)) {
+                return serviceNode;
+            }
+        }
+
+        return getVersionedControllerService(group.getParent(), versionedComponentId);
+    }
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/mapping/InstantiatedConnectableComponent.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/mapping/InstantiatedConnectableComponent.java
index 81921b2..9bb0659 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/mapping/InstantiatedConnectableComponent.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/mapping/InstantiatedConnectableComponent.java
@@ -29,7 +29,7 @@ public class InstantiatedConnectableComponent extends ConnectableComponent imple
     }
 
     @Override
-    public String getInstanceId() {
+    public String getInstanceIdentifier() {
         return instanceId;
     }
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/mapping/InstantiatedVersionedComponent.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/mapping/InstantiatedVersionedComponent.java
index e655c16..15fbfc7 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/mapping/InstantiatedVersionedComponent.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/mapping/InstantiatedVersionedComponent.java
@@ -21,9 +21,7 @@ import javax.xml.bind.annotation.XmlTransient;
 
 public interface InstantiatedVersionedComponent {
 
-    // mark transient so fields are ignored when serializing all versioned component types
-    @XmlTransient
-    String getInstanceId();
+    String getInstanceIdentifier();
 
     @XmlTransient
     String getInstanceGroupId();
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/mapping/InstantiatedVersionedConnection.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/mapping/InstantiatedVersionedConnection.java
index 3fdd95a..86a8503 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/mapping/InstantiatedVersionedConnection.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/mapping/InstantiatedVersionedConnection.java
@@ -29,7 +29,7 @@ public class InstantiatedVersionedConnection extends VersionedConnection impleme
     }
 
     @Override
-    public String getInstanceId() {
+    public String getInstanceIdentifier() {
         return instanceId;
     }
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/mapping/InstantiatedVersionedControllerService.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/mapping/InstantiatedVersionedControllerService.java
index 71b35be..6b4ec6d 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/mapping/InstantiatedVersionedControllerService.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/mapping/InstantiatedVersionedControllerService.java
@@ -29,7 +29,7 @@ public class InstantiatedVersionedControllerService extends VersionedControllerS
     }
 
     @Override
-    public String getInstanceId() {
+    public String getInstanceIdentifier() {
         return instanceId;
     }
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/mapping/InstantiatedVersionedFunnel.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/mapping/InstantiatedVersionedFunnel.java
index 0ed2f07..f92488c 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/mapping/InstantiatedVersionedFunnel.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/mapping/InstantiatedVersionedFunnel.java
@@ -29,7 +29,7 @@ public class InstantiatedVersionedFunnel extends VersionedFunnel implements Inst
     }
 
     @Override
-    public String getInstanceId() {
+    public String getInstanceIdentifier() {
         return instanceId;
     }
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/mapping/InstantiatedVersionedLabel.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/mapping/InstantiatedVersionedLabel.java
index 1e70144..e26fe6b 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/mapping/InstantiatedVersionedLabel.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/mapping/InstantiatedVersionedLabel.java
@@ -29,7 +29,7 @@ public class InstantiatedVersionedLabel extends VersionedLabel implements Instan
     }
 
     @Override
-    public String getInstanceId() {
+    public String getInstanceIdentifier() {
         return instanceId;
     }
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/mapping/InstantiatedVersionedPort.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/mapping/InstantiatedVersionedPort.java
index e601e99..8736505 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/mapping/InstantiatedVersionedPort.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/mapping/InstantiatedVersionedPort.java
@@ -29,7 +29,7 @@ public class InstantiatedVersionedPort extends VersionedPort implements Instanti
     }
 
     @Override
-    public String getInstanceId() {
+    public String getInstanceIdentifier() {
         return instanceId;
     }
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/mapping/InstantiatedVersionedProcessGroup.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/mapping/InstantiatedVersionedProcessGroup.java
index 3eeac75..b1d1a13 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/mapping/InstantiatedVersionedProcessGroup.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/mapping/InstantiatedVersionedProcessGroup.java
@@ -35,7 +35,7 @@ public class InstantiatedVersionedProcessGroup extends VersionedProcessGroup imp
     }
 
     @Override
-    public String getInstanceId() {
+    public String getInstanceIdentifier() {
         return instanceId;
     }
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/mapping/InstantiatedVersionedProcessor.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/mapping/InstantiatedVersionedProcessor.java
index f00692f..eedd0ff 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/mapping/InstantiatedVersionedProcessor.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/mapping/InstantiatedVersionedProcessor.java
@@ -29,7 +29,7 @@ public class InstantiatedVersionedProcessor extends VersionedProcessor implement
     }
 
     @Override
-    public String getInstanceId() {
+    public String getInstanceIdentifier() {
         return instanceId;
     }
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/mapping/InstantiatedVersionedRemoteGroupPort.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/mapping/InstantiatedVersionedRemoteGroupPort.java
index e3bfa46..6d70e0b 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/mapping/InstantiatedVersionedRemoteGroupPort.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/mapping/InstantiatedVersionedRemoteGroupPort.java
@@ -29,7 +29,7 @@ public class InstantiatedVersionedRemoteGroupPort extends VersionedRemoteGroupPo
     }
 
     @Override
-    public String getInstanceId() {
+    public String getInstanceIdentifier() {
         return instanceId;
     }
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/mapping/InstantiatedVersionedRemoteProcessGroup.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/mapping/InstantiatedVersionedRemoteProcessGroup.java
index 65dbef1..7bd3993 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/mapping/InstantiatedVersionedRemoteProcessGroup.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/mapping/InstantiatedVersionedRemoteProcessGroup.java
@@ -29,7 +29,7 @@ public class InstantiatedVersionedRemoteProcessGroup extends VersionedRemoteProc
     }
 
     @Override
-    public String getInstanceId() {
+    public String getInstanceIdentifier() {
         return instanceId;
     }
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/mapping/NiFiRegistryFlowMapper.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/mapping/NiFiRegistryFlowMapper.java
index 889f3f1..886916e 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/mapping/NiFiRegistryFlowMapper.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/mapping/NiFiRegistryFlowMapper.java
@@ -30,52 +30,55 @@ import org.apache.nifi.controller.ComponentNode;
 import org.apache.nifi.controller.ControllerService;
 import org.apache.nifi.controller.ProcessorNode;
 import org.apache.nifi.controller.PropertyConfiguration;
+import org.apache.nifi.controller.ReportingTaskNode;
 import org.apache.nifi.controller.ScheduledState;
 import org.apache.nifi.controller.label.Label;
 import org.apache.nifi.controller.queue.FlowFileQueue;
 import org.apache.nifi.controller.service.ControllerServiceNode;
 import org.apache.nifi.controller.service.ControllerServiceProvider;
-import org.apache.nifi.groups.ProcessGroup;
-import org.apache.nifi.groups.RemoteProcessGroup;
-import org.apache.nifi.nar.ExtensionManager;
-import org.apache.nifi.parameter.Parameter;
-import org.apache.nifi.parameter.ParameterContext;
-import org.apache.nifi.parameter.ParameterDescriptor;
-import org.apache.nifi.processor.Relationship;
-import org.apache.nifi.registry.VariableDescriptor;
 import org.apache.nifi.flow.BatchSize;
 import org.apache.nifi.flow.Bundle;
 import org.apache.nifi.flow.ComponentType;
 import org.apache.nifi.flow.ConnectableComponent;
 import org.apache.nifi.flow.ConnectableComponentType;
 import org.apache.nifi.flow.ControllerServiceAPI;
-import org.apache.nifi.registry.flow.ExternalControllerServiceReference;
-import org.apache.nifi.registry.flow.FlowRegistry;
-import org.apache.nifi.registry.flow.FlowRegistryClient;
 import org.apache.nifi.flow.PortType;
 import org.apache.nifi.flow.Position;
-import org.apache.nifi.registry.flow.VersionControlInformation;
 import org.apache.nifi.flow.VersionedConnection;
 import org.apache.nifi.flow.VersionedControllerService;
 import org.apache.nifi.flow.VersionedFlowCoordinates;
 import org.apache.nifi.flow.VersionedFunnel;
 import org.apache.nifi.flow.VersionedLabel;
-import org.apache.nifi.registry.flow.VersionedParameter;
-import org.apache.nifi.registry.flow.VersionedParameterContext;
 import org.apache.nifi.flow.VersionedPort;
 import org.apache.nifi.flow.VersionedProcessGroup;
 import org.apache.nifi.flow.VersionedProcessor;
 import org.apache.nifi.flow.VersionedPropertyDescriptor;
 import org.apache.nifi.flow.VersionedRemoteGroupPort;
 import org.apache.nifi.flow.VersionedRemoteProcessGroup;
+import org.apache.nifi.flow.VersionedReportingTask;
 import org.apache.nifi.flow.VersionedResourceCardinality;
 import org.apache.nifi.flow.VersionedResourceDefinition;
 import org.apache.nifi.flow.VersionedResourceType;
+import org.apache.nifi.groups.ProcessGroup;
+import org.apache.nifi.groups.RemoteProcessGroup;
+import org.apache.nifi.nar.ExtensionManager;
+import org.apache.nifi.parameter.Parameter;
+import org.apache.nifi.parameter.ParameterContext;
+import org.apache.nifi.parameter.ParameterDescriptor;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.registry.VariableDescriptor;
+import org.apache.nifi.registry.flow.ExternalControllerServiceReference;
+import org.apache.nifi.registry.flow.FlowRegistry;
+import org.apache.nifi.registry.flow.FlowRegistryClient;
+import org.apache.nifi.registry.flow.VersionControlInformation;
+import org.apache.nifi.registry.flow.VersionedParameter;
+import org.apache.nifi.registry.flow.VersionedParameterContext;
 import org.apache.nifi.remote.PublicPort;
 import org.apache.nifi.remote.RemoteGroupPort;
 
 import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.LinkedHashSet;
@@ -91,8 +94,11 @@ import java.util.stream.Collectors;
 
 
 public class NiFiRegistryFlowMapper {
+    private static final String ENCRYPTED_PREFIX = "enc{";
+    private static final String ENCRYPTED_SUFFIX = "}";
 
     private final ExtensionManager extensionManager;
+    private final FlowMappingOptions flowMappingOptions;
 
     // We need to keep a mapping of component id to versionedComponentId as we transform these objects. This way, when
     // we call #mapConnectable, instead of generating a new UUID for the ConnectableComponent, we can lookup the 'versioned'
@@ -101,7 +107,12 @@ public class NiFiRegistryFlowMapper {
     private Map<String, String> versionedComponentIds = new HashMap<>();
 
     public NiFiRegistryFlowMapper(final ExtensionManager extensionManager) {
+        this(extensionManager, FlowMappingOptions.DEFAULT_OPTIONS);
+    }
+
+    public NiFiRegistryFlowMapper(final ExtensionManager extensionManager, final FlowMappingOptions flowMappingOptions) {
         this.extensionManager = extensionManager;
+        this.flowMappingOptions = flowMappingOptions;
     }
 
     /**
@@ -191,6 +202,9 @@ public class NiFiRegistryFlowMapper {
 
         final InstantiatedVersionedProcessGroup versionedGroup = new InstantiatedVersionedProcessGroup(group.getIdentifier(), group.getProcessGroupIdentifier());
         versionedGroup.setIdentifier(getId(group.getVersionedComponentId(), group.getIdentifier()));
+        if (flowMappingOptions.isMapInstanceIdentifiers()) {
+            versionedGroup.setInstanceIdentifier(group.getIdentifier());
+        }
         versionedGroup.setGroupIdentifier(getGroupId(group.getProcessGroupIdentifier()));
         versionedGroup.setName(group.getName());
         versionedGroup.setComments(group.getComments());
@@ -299,13 +313,7 @@ public class NiFiRegistryFlowMapper {
     }
 
     private String getId(final Optional<String> currentVersionedId, final String componentId) {
-        final String versionedId;
-        if (currentVersionedId.isPresent()) {
-            versionedId = currentVersionedId.get();
-        } else {
-            versionedId = generateVersionedComponentId(componentId);
-        }
-
+        final String versionedId = flowMappingOptions.getComponentIdLookup().getComponentId(currentVersionedId, componentId);
         versionedComponentIds.put(componentId, versionedId);
         return versionedId;
     }
@@ -321,17 +329,13 @@ public class NiFiRegistryFlowMapper {
         return UUID.nameUUIDFromBytes(componentId.getBytes(StandardCharsets.UTF_8)).toString();
     }
 
-    private <E extends Exception> String getIdOrThrow(final Optional<String> currentVersionedId, final String componentId, final Supplier<E> exceptionSupplier) throws E {
-        if (currentVersionedId.isPresent()) {
-            return currentVersionedId.get();
-        } else {
-            final String resolved = versionedComponentIds.get(componentId);
-            if (resolved == null) {
-                throw exceptionSupplier.get();
-            }
-
-            return resolved;
+    private <E extends Exception> String getIdOrThrow(final String componentId, final Supplier<E> exceptionSupplier) throws E {
+        final String resolved = versionedComponentIds.get(componentId);
+        if (resolved == null) {
+            throw exceptionSupplier.get();
         }
+
+        return resolved;
     }
 
 
@@ -344,6 +348,9 @@ public class NiFiRegistryFlowMapper {
 
         final VersionedConnection versionedConnection = new InstantiatedVersionedConnection(connection.getIdentifier(), connection.getProcessGroup().getIdentifier());
         versionedConnection.setIdentifier(getId(connection.getVersionedComponentId(), connection.getIdentifier()));
+        if (flowMappingOptions.isMapInstanceIdentifiers()) {
+            versionedConnection.setInstanceIdentifier(connection.getIdentifier());
+        }
         versionedConnection.setGroupIdentifier(getGroupId(connection.getProcessGroup().getIdentifier()));
         versionedConnection.setName(connection.getName());
         versionedConnection.setBackPressureDataSizeThreshold(queue.getBackPressureDataSizeThreshold());
@@ -372,9 +379,12 @@ public class NiFiRegistryFlowMapper {
     public ConnectableComponent mapConnectable(final Connectable connectable) {
         final ConnectableComponent component = new InstantiatedConnectableComponent(connectable.getIdentifier(), connectable.getProcessGroupIdentifier());
 
-        final String versionedId = getIdOrThrow(connectable.getVersionedComponentId(), connectable.getIdentifier(),
+        final String versionedId = getIdOrThrow(connectable.getIdentifier(),
             () -> new IllegalArgumentException("Unable to map Connectable Component with identifier " + connectable.getIdentifier() + " to any version-controlled component"));
         component.setId(versionedId);
+        if (flowMappingOptions.isMapInstanceIdentifiers()) {
+            component.setInstanceIdentifier(connectable.getIdentifier());
+        }
 
         component.setComments(connectable.getComments());
 
@@ -382,12 +392,11 @@ public class NiFiRegistryFlowMapper {
         if (connectable instanceof RemoteGroupPort) {
             final RemoteGroupPort port = (RemoteGroupPort) connectable;
             final RemoteProcessGroup rpg = port.getRemoteProcessGroup();
-            final Optional<String> rpgVersionedId = rpg.getVersionedComponentId();
-            groupId = getIdOrThrow(rpgVersionedId, rpg.getIdentifier(),
+            groupId = getIdOrThrow(rpg.getIdentifier(),
                 () -> new IllegalArgumentException("Unable to find the Versioned Component ID for Remote Process Group that " + connectable + " belongs to"));
 
         } else {
-            groupId = getIdOrThrow(connectable.getProcessGroup().getVersionedComponentId(), connectable.getProcessGroupIdentifier(),
+            groupId = getIdOrThrow(connectable.getProcessGroupIdentifier(),
                 () -> new IllegalArgumentException("Unable to find the Versioned Component ID for the Process Group that " + connectable + " belongs to"));
         }
 
@@ -398,10 +407,35 @@ public class NiFiRegistryFlowMapper {
         return component;
     }
 
+    public VersionedReportingTask mapReportingTask(final ReportingTaskNode taskNode, final ControllerServiceProvider serviceProvider) {
+        final VersionedReportingTask versionedTask = new VersionedReportingTask();
+        versionedTask.setIdentifier(taskNode.getIdentifier());
+        if (flowMappingOptions.isMapInstanceIdentifiers()) {
+            versionedTask.setInstanceIdentifier(taskNode.getIdentifier());
+        }
+        versionedTask.setAnnotationData(taskNode.getAnnotationData());
+        versionedTask.setBundle(mapBundle(taskNode.getBundleCoordinate()));
+        versionedTask.setComments(taskNode.getComments());
+        versionedTask.setComponentType(ComponentType.REPORTING_TASK);
+        versionedTask.setName(taskNode.getName());
+
+        versionedTask.setProperties(mapProperties(taskNode, serviceProvider));
+        versionedTask.setPropertyDescriptors(mapPropertyDescriptors(taskNode, serviceProvider, Collections.emptySet(), Collections.emptyMap()));
+        versionedTask.setSchedulingPeriod(taskNode.getSchedulingPeriod());
+        versionedTask.setSchedulingStrategy(taskNode.getSchedulingStrategy().name());
+        versionedTask.setType(taskNode.getCanonicalClassName());
+        versionedTask.setScheduledState(flowMappingOptions.getStateLookup().getState(taskNode));
+
+        return versionedTask;
+    }
+
     public VersionedControllerService mapControllerService(final ControllerServiceNode controllerService, final ControllerServiceProvider serviceProvider, final Set<String> includedGroupIds,
                                                            final Map<String, ExternalControllerServiceReference> externalControllerServiceReferences) {
         final VersionedControllerService versionedService = new InstantiatedVersionedControllerService(controllerService.getIdentifier(), controllerService.getProcessGroupIdentifier());
         versionedService.setIdentifier(getId(controllerService.getVersionedComponentId(), controllerService.getIdentifier()));
+        if (flowMappingOptions.isMapInstanceIdentifiers()) {
+            versionedService.setInstanceIdentifier(controllerService.getIdentifier());
+        }
         versionedService.setGroupIdentifier(getGroupId(controllerService.getProcessGroupIdentifier()));
         versionedService.setName(controllerService.getName());
         versionedService.setAnnotationData(controllerService.getAnnotationData());
@@ -412,6 +446,7 @@ public class NiFiRegistryFlowMapper {
         versionedService.setProperties(mapProperties(controllerService, serviceProvider));
         versionedService.setPropertyDescriptors(mapPropertyDescriptors(controllerService, serviceProvider, includedGroupIds, externalControllerServiceReferences));
         versionedService.setType(controllerService.getCanonicalClassName());
+        versionedService.setScheduledState(flowMappingOptions.getStateLookup().getState(controllerService));
 
         return versionedService;
     }
@@ -427,7 +462,7 @@ public class NiFiRegistryFlowMapper {
                     value = property.getDefaultValue();
                 }
 
-                if (value != null && property.getControllerServiceDefinition() != null) {
+                if (value != null && property.getControllerServiceDefinition() != null && flowMappingOptions.isMapControllerServiceReferencesToVersionedId()) {
                     // Property references a Controller Service. Instead of storing the existing value, we want
                     // to store the Versioned Component ID of the service.
                     final ControllerServiceNode controllerService = serviceProvider.getControllerServiceNode(value);
@@ -436,17 +471,40 @@ public class NiFiRegistryFlowMapper {
                     }
                 }
 
+                if (property.isSensitive()) {
+                    value = encrypt(value);
+                }
+
                 mapped.put(property.getName(), value);
             });
 
         return mapped;
     }
 
+    private String encrypt(final String value) {
+        if (value == null) {
+            return null;
+        }
+
+        final SensitiveValueEncryptor encryptor = flowMappingOptions.getSensitiveValueEncryptor();
+        if (encryptor == null) {
+            // This will happen only if the given property is mappable, which means that it is a parameter reference.
+            return value;
+        }
+
+        final String encrypted = encryptor.encrypt(value);
+        return ENCRYPTED_PREFIX + encrypted + ENCRYPTED_SUFFIX;
+    }
+
     private boolean isMappable(final PropertyDescriptor propertyDescriptor, final PropertyConfiguration propertyConfiguration) {
         if (!propertyDescriptor.isSensitive()) { // If the property is not sensitive, it can be mapped.
             return true;
         }
 
+        if (flowMappingOptions.isMapSensitiveConfiguration()) {
+            return true;
+        }
+
         if (propertyConfiguration == null) {
             return false;
         }
@@ -459,6 +517,11 @@ public class NiFiRegistryFlowMapper {
 
     private Map<String, VersionedPropertyDescriptor> mapPropertyDescriptors(final ComponentNode component, final ControllerServiceProvider serviceProvider, final Set<String> includedGroupIds,
                                                                             final Map<String, ExternalControllerServiceReference> externalControllerServiceReferences) {
+
+        if (!flowMappingOptions.isMapPropertyDescriptors()) {
+            return Collections.emptyMap();
+        }
+
         final Map<String, VersionedPropertyDescriptor> descriptors = new HashMap<>();
         for (final PropertyDescriptor descriptor : component.getProperties().keySet()) {
             final VersionedPropertyDescriptor versionedDescriptor = new VersionedPropertyDescriptor();
@@ -553,6 +616,9 @@ public class NiFiRegistryFlowMapper {
     public VersionedFunnel mapFunnel(final Funnel funnel) {
         final VersionedFunnel versionedFunnel = new InstantiatedVersionedFunnel(funnel.getIdentifier(), funnel.getProcessGroupIdentifier());
         versionedFunnel.setIdentifier(getId(funnel.getVersionedComponentId(), funnel.getIdentifier()));
+        if (flowMappingOptions.isMapInstanceIdentifiers()) {
+            versionedFunnel.setInstanceIdentifier(funnel.getIdentifier());
+        }
         versionedFunnel.setGroupIdentifier(getGroupId(funnel.getProcessGroupIdentifier()));
         versionedFunnel.setPosition(mapPosition(funnel.getPosition()));
 
@@ -562,6 +628,9 @@ public class NiFiRegistryFlowMapper {
     public VersionedLabel mapLabel(final Label label) {
         final VersionedLabel versionedLabel = new InstantiatedVersionedLabel(label.getIdentifier(), label.getProcessGroupIdentifier());
         versionedLabel.setIdentifier(getId(label.getVersionedComponentId(), label.getIdentifier()));
+        if (flowMappingOptions.isMapInstanceIdentifiers()) {
+            versionedLabel.setInstanceIdentifier(label.getIdentifier());
+        }
         versionedLabel.setGroupIdentifier(getGroupId(label.getProcessGroupIdentifier()));
         versionedLabel.setHeight(label.getSize().getHeight());
         versionedLabel.setWidth(label.getSize().getWidth());
@@ -575,6 +644,9 @@ public class NiFiRegistryFlowMapper {
     public VersionedPort mapPort(final Port port) {
         final VersionedPort versionedPort = new InstantiatedVersionedPort(port.getIdentifier(), port.getProcessGroupIdentifier());
         versionedPort.setIdentifier(getId(port.getVersionedComponentId(), port.getIdentifier()));
+        if (flowMappingOptions.isMapInstanceIdentifiers()) {
+            versionedPort.setInstanceIdentifier(port.getIdentifier());
+        }
         versionedPort.setGroupIdentifier(getGroupId(port.getProcessGroupIdentifier()));
         versionedPort.setComments(port.getComments());
         versionedPort.setConcurrentlySchedulableTaskCount(port.getMaxConcurrentTasks());
@@ -589,6 +661,8 @@ public class NiFiRegistryFlowMapper {
             versionedPort.setAllowRemoteAccess(false);
         }
 
+        versionedPort.setScheduledState(flowMappingOptions.getStateLookup().getState(port));
+
         return versionedPort;
     }
 
@@ -603,6 +677,9 @@ public class NiFiRegistryFlowMapper {
                                            final Map<String, ExternalControllerServiceReference> externalControllerServiceReferences) {
         final VersionedProcessor processor = new InstantiatedVersionedProcessor(procNode.getIdentifier(), procNode.getProcessGroupIdentifier());
         processor.setIdentifier(getId(procNode.getVersionedComponentId(), procNode.getIdentifier()));
+        if (flowMappingOptions.isMapInstanceIdentifiers()) {
+            processor.setInstanceIdentifier(procNode.getIdentifier());
+        }
         processor.setGroupIdentifier(getGroupId(procNode.getProcessGroupIdentifier()));
         processor.setType(procNode.getCanonicalClassName());
         processor.setAnnotationData(procNode.getAnnotationData());
@@ -622,7 +699,7 @@ public class NiFiRegistryFlowMapper {
         processor.setSchedulingStrategy(procNode.getSchedulingStrategy().name());
         processor.setStyle(procNode.getStyle());
         processor.setYieldDuration(procNode.getYieldPeriod());
-        processor.setScheduledState(mapScheduledState(procNode.getScheduledState()));
+        processor.setScheduledState(flowMappingOptions.getStateLookup().getState(procNode));
 
         return processor;
     }
@@ -630,6 +707,9 @@ public class NiFiRegistryFlowMapper {
     public VersionedRemoteProcessGroup mapRemoteProcessGroup(final RemoteProcessGroup remoteGroup) {
         final VersionedRemoteProcessGroup rpg = new InstantiatedVersionedRemoteProcessGroup(remoteGroup.getIdentifier(), remoteGroup.getProcessGroupIdentifier());
         rpg.setIdentifier(getId(remoteGroup.getVersionedComponentId(), remoteGroup.getIdentifier()));
+        if (flowMappingOptions.isMapInstanceIdentifiers()) {
+            rpg.setInstanceIdentifier(remoteGroup.getIdentifier());
+        }
         rpg.setGroupIdentifier(getGroupId(remoteGroup.getProcessGroupIdentifier()));
         rpg.setComments(remoteGroup.getComments());
         rpg.setCommunicationsTimeout(remoteGroup.getCommunicationsTimeout());
@@ -655,6 +735,9 @@ public class NiFiRegistryFlowMapper {
     public VersionedRemoteGroupPort mapRemotePort(final RemoteGroupPort remotePort, final ComponentType componentType) {
         final VersionedRemoteGroupPort port = new InstantiatedVersionedRemoteGroupPort(remotePort.getIdentifier(), remotePort.getRemoteProcessGroup().getIdentifier());
         port.setIdentifier(getId(remotePort.getVersionedComponentId(), remotePort.getIdentifier()));
+        if (flowMappingOptions.isMapInstanceIdentifiers()) {
+            port.setInstanceIdentifier(remotePort.getIdentifier());
+        }
         port.setGroupIdentifier(getGroupId(remotePort.getRemoteProcessGroup().getIdentifier()));
         port.setComments(remotePort.getComments());
         port.setConcurrentlySchedulableTaskCount(remotePort.getMaxConcurrentTasks());
@@ -664,7 +747,8 @@ public class NiFiRegistryFlowMapper {
         port.setBatchSize(mapBatchSettings(remotePort));
         port.setTargetId(remotePort.getTargetIdentifier());
         port.setComponentType(componentType);
-        port.setScheduledState(mapScheduledState(remotePort.getScheduledState()));
+        port.setScheduledState(flowMappingOptions.getStateLookup().getState(remotePort));
+
         return port;
     }
 
@@ -676,6 +760,28 @@ public class NiFiRegistryFlowMapper {
         return batchSize;
     }
 
+    public VersionedParameterContext mapParameterContext(final ParameterContext parameterContext) {
+        final Set<VersionedParameter> versionedParameters = new HashSet<>();
+        for (final Parameter parameter : parameterContext.getParameters().values()) {
+            final VersionedParameter versionedParameter = mapParameter(parameter);
+            versionedParameters.add(versionedParameter);
+        }
+
+        final VersionedParameterContext versionedParameterContext = new VersionedParameterContext();
+        versionedParameterContext.setDescription(parameterContext.getDescription());
+        versionedParameterContext.setName(parameterContext.getName());
+        versionedParameterContext.setParameters(versionedParameters);
+        final String versionedContextId = flowMappingOptions.getComponentIdLookup().getComponentId(Optional.empty(), parameterContext.getIdentifier());
+        versionedParameterContext.setIdentifier(versionedContextId);
+        versionedParameterContext.setInheritedParameterContexts(parameterContext.getInheritedParameterContextNames());
+
+        if (flowMappingOptions.isMapInstanceIdentifiers()) {
+            versionedParameterContext.setInstanceIdentifier(parameterContext.getIdentifier());
+        }
+
+        return versionedParameterContext;
+    }
+
     public Map<String, VersionedParameterContext> mapParameterContexts(final ProcessGroup processGroup,
                                                                        final boolean mapDescendantVersionedFlows) {
         // cannot use a set to enforce uniqueness of parameter contexts because VersionedParameterContext in the
@@ -728,7 +834,20 @@ public class NiFiRegistryFlowMapper {
         versionedParameter.setDescription(descriptor.getDescription());
         versionedParameter.setName(descriptor.getName());
         versionedParameter.setSensitive(descriptor.isSensitive());
-        versionedParameter.setValue(descriptor.isSensitive() ? null : parameter.getValue());
+
+        final boolean mapParameterValue = flowMappingOptions.isMapSensitiveConfiguration() || !descriptor.isSensitive();
+        final String parameterValue;
+        if (mapParameterValue) {
+            if (descriptor.isSensitive()) {
+                parameterValue = encrypt(parameter.getValue());
+            } else {
+                parameterValue = parameter.getValue();
+            }
+        } else {
+            parameterValue = null;
+        }
+
+        versionedParameter.setValue(parameterValue);
         return versionedParameter;
     }
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/mapping/StandardComparableDataFlow.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/mapping/StandardComparableDataFlow.java
index eb37489..2f210a0 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/mapping/StandardComparableDataFlow.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/mapping/StandardComparableDataFlow.java
@@ -17,16 +17,34 @@
 
 package org.apache.nifi.registry.flow.mapping;
 
+import org.apache.nifi.flow.VersionedControllerService;
 import org.apache.nifi.flow.VersionedProcessGroup;
+import org.apache.nifi.flow.VersionedReportingTask;
+import org.apache.nifi.registry.flow.VersionedParameterContext;
 import org.apache.nifi.registry.flow.diff.ComparableDataFlow;
 
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+
 public class StandardComparableDataFlow implements ComparableDataFlow {
     private final String name;
     private final VersionedProcessGroup contents;
+    private final Set<VersionedControllerService> controllerLevelServices;
+    private final Set<VersionedReportingTask> reportingTasks;
+    private final Set<VersionedParameterContext> parameterContexts;
 
     public StandardComparableDataFlow(final String name, final VersionedProcessGroup contents) {
+        this(name, contents, Collections.emptySet(), Collections.emptySet(), Collections.emptySet());
+    }
+
+    public StandardComparableDataFlow(final String name, final VersionedProcessGroup contents, final Set<VersionedControllerService> controllerLevelServices,
+                                      final Set<VersionedReportingTask> reportingTasks, final Set<VersionedParameterContext> parameterContexts) {
         this.name = name;
         this.contents = contents;
+        this.controllerLevelServices = controllerLevelServices == null ? Collections.emptySet() : new HashSet<>(controllerLevelServices);
+        this.reportingTasks = reportingTasks == null ? Collections.emptySet() : new HashSet<>(reportingTasks);
+        this.parameterContexts = parameterContexts == null ? Collections.emptySet() : new HashSet<>(parameterContexts);
     }
 
     @Override
@@ -39,4 +57,18 @@ public class StandardComparableDataFlow implements ComparableDataFlow {
         return contents;
     }
 
+    @Override
+    public Set<VersionedControllerService> getControllerLevelServices() {
+        return controllerLevelServices;
+    }
+
+    @Override
+    public Set<VersionedReportingTask> getReportingTasks() {
+        return reportingTasks;
+    }
+
+    @Override
+    public Set<VersionedParameterContext> getParameterContexts() {
+        return parameterContexts;
+    }
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/util/FlowDifferenceFilters.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/util/FlowDifferenceFilters.java
index f0ec288..61e528f 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/util/FlowDifferenceFilters.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/util/FlowDifferenceFilters.java
@@ -21,7 +21,6 @@ import org.apache.nifi.controller.ComponentNode;
 import org.apache.nifi.controller.ProcessorNode;
 import org.apache.nifi.controller.flow.FlowManager;
 import org.apache.nifi.controller.service.ControllerServiceNode;
-import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.flow.ComponentType;
 import org.apache.nifi.flow.ScheduledState;
 import org.apache.nifi.flow.VersionedComponent;
@@ -30,6 +29,7 @@ import org.apache.nifi.flow.VersionedFlowCoordinates;
 import org.apache.nifi.flow.VersionedPort;
 import org.apache.nifi.flow.VersionedProcessGroup;
 import org.apache.nifi.flow.VersionedProcessor;
+import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.registry.flow.diff.DifferenceType;
 import org.apache.nifi.registry.flow.diff.FlowDifference;
 import org.apache.nifi.registry.flow.mapping.InstantiatedVersionedComponent;
@@ -46,6 +46,28 @@ import java.util.function.Predicate;
 public class FlowDifferenceFilters {
 
     /**
+     * Determines whether or not the Flow Difference depicts an environmental change. I.e., a change that is expected to happen from environment to environment,
+     * and which should be considered a "local modification" to a dataflow after a flow has been imported from a flow registry
+     * @param difference the Flow Difference to consider
+     * @param localGroup a mapping of the local Process Group
+     * @param flowManager the Flow Manager
+     * @return <code>true</code> if the change is an environment-specific change, <code>false</code> otherwise
+     */
+    public static boolean isEnvironmentalChange(final FlowDifference difference, final VersionedProcessGroup localGroup, final FlowManager flowManager) {
+        return difference.getDifferenceType() == DifferenceType.BUNDLE_CHANGED
+            || isVariableValueChange(difference)
+            || isRpgUrlChange(difference)
+            || isAddedOrRemovedRemotePort(difference)
+            || isPublicPortNameChange(difference)
+            || isIgnorableVersionedFlowCoordinateChange(difference)
+            || isNewPropertyWithDefaultValue(difference, flowManager)
+            || isNewRelationshipAutoTerminatedAndDefaulted(difference, localGroup, flowManager)
+            || isScheduledStateNew(difference)
+            || isLocalScheduleStateChange(difference)
+            || isPropertyMissingFromGhostComponent(difference, flowManager);
+    }
+
+    /**
      * Predicate that returns true if the difference is NOT a name change on a public port (i.e. VersionedPort that allows remote access).
      */
     public static Predicate<FlowDifference> FILTER_PUBLIC_PORT_NAME_CHANGES = (fd) -> !isPublicPortNameChange(fd);
@@ -131,11 +153,11 @@ public class FlowDifferenceFilters {
 
         if (componentB instanceof InstantiatedVersionedProcessor) {
             final InstantiatedVersionedProcessor instantiatedProcessor = (InstantiatedVersionedProcessor) componentB;
-            final ProcessorNode processorNode = flowManager.getProcessorNode(instantiatedProcessor.getInstanceId());
+            final ProcessorNode processorNode = flowManager.getProcessorNode(instantiatedProcessor.getInstanceIdentifier());
             return isNewPropertyWithDefaultValue(fd, processorNode);
         } else if (componentB instanceof InstantiatedVersionedControllerService) {
             final InstantiatedVersionedControllerService instantiatedControllerService = (InstantiatedVersionedControllerService) componentB;
-            final ControllerServiceNode controllerService = flowManager.getControllerServiceNode(instantiatedControllerService.getInstanceId());
+            final ControllerServiceNode controllerService = flowManager.getControllerServiceNode(instantiatedControllerService.getInstanceIdentifier());
             return isNewPropertyWithDefaultValue(fd, controllerService);
         }
 
@@ -181,6 +203,42 @@ public class FlowDifferenceFilters {
         return false;
     }
 
+    /**
+     * @return <code>true</code> if the Flow Difference shows a processor/port transitioning between stopped/running or a controller service transitioning
+     * between enabled/disabled. These are a normal part of the flow lifecycle and don't represent changes to the flow itself.
+     */
+    public static boolean isLocalScheduleStateChange(final FlowDifference fd) {
+        if (fd.getDifferenceType() != DifferenceType.SCHEDULED_STATE_CHANGED) {
+            return false;
+        }
+
+        if (fd.getComponentA().getComponentType() == ComponentType.CONTROLLER_SERVICE) {
+            return true;
+        }
+
+        final String scheduledStateB = String.valueOf(fd.getValueB());
+        final String scheduledStateA = String.valueOf(fd.getValueA());
+
+        // If transitioned from 'STOPPED' or 'ENABLED' to 'RUNNING', this is a 'local' schedule State Change.
+        // Because of this, it won't be a considered a difference between the local, running flow, and a versioned flow
+        if ("RUNNING".equals(scheduledStateB) && ("STOPPED".equals(scheduledStateA) || "ENABLED".equals(scheduledStateA))) {
+            return true;
+        }
+        if ("RUNNING".equals(scheduledStateA) && ("STOPPED".equals(scheduledStateB) || "ENABLED".equals(scheduledStateB))) {
+            return true;
+        }
+
+        return false;
+    }
+
+    public static boolean isVariableValueChange(final FlowDifference flowDifference) {
+        return flowDifference.getDifferenceType() == DifferenceType.VARIABLE_CHANGED;
+    }
+
+    public static boolean isRpgUrlChange(final FlowDifference flowDifference) {
+        return flowDifference.getDifferenceType() == DifferenceType.RPG_URL_CHANGED;
+    }
+
     public static boolean isNewRelationshipAutoTerminatedAndDefaulted(final FlowDifference fd, final VersionedProcessGroup processGroup, final FlowManager flowManager) {
         if (fd.getDifferenceType() != DifferenceType.AUTO_TERMINATED_RELATIONSHIPS_CHANGED) {
             return false;
@@ -207,7 +265,7 @@ public class FlowDifferenceFilters {
         }
 
         final InstantiatedVersionedProcessor instantiatedVersionedProcessor = (InstantiatedVersionedProcessor) processorB;
-        final ProcessorNode processorNode = flowManager.getProcessorNode(instantiatedVersionedProcessor.getInstanceId());
+        final ProcessorNode processorNode = flowManager.getProcessorNode(instantiatedVersionedProcessor.getInstanceIdentifier());
         if (processorNode == null) {
             return false;
         }
@@ -239,6 +297,51 @@ public class FlowDifferenceFilters {
     }
 
     /**
+     * If a property is removed from a ghosted component, we may want to ignore it. This is because all properties will be considered sensitive for
+     * a ghosted component and as a result, the property map may not be populated with its property value, resulting in an indication that the property
+     * is missing when it is not.
+     */
+    public static boolean isPropertyMissingFromGhostComponent(final FlowDifference difference, final FlowManager flowManager) {
+        if (difference.getDifferenceType() != DifferenceType.PROPERTY_REMOVED) {
+            return false;
+        }
+
+        final Optional<String> fieldName = difference.getFieldName();
+        if (!fieldName.isPresent()) {
+            return false;
+        }
+
+        final VersionedComponent componentB = difference.getComponentB();
+        if (componentB instanceof InstantiatedVersionedProcessor) {
+            final ProcessorNode procNode = flowManager.getProcessorNode(componentB.getInstanceIdentifier());
+            return procNode.isExtensionMissing() && isPropertyPresent(procNode, difference);
+        }
+
+        if (componentB instanceof InstantiatedVersionedControllerService) {
+            final ControllerServiceNode serviceNode = flowManager.getControllerServiceNode(componentB.getInstanceIdentifier());
+            return serviceNode.isExtensionMissing() && isPropertyPresent(serviceNode, difference);
+        }
+
+        return false;
+    }
+
+    private static boolean isPropertyPresent(final ComponentNode componentNode, final FlowDifference difference) {
+        if (componentNode == null) {
+            return false;
+        }
+
+        final Optional<String> fieldNameOptional = difference.getFieldName();
+        if (!fieldNameOptional.isPresent()) {
+            return false;
+        }
+
+        // Check if a value is configured. If any value is configured, then the property is not actually missing.
+        final PropertyDescriptor descriptor = componentNode.getPropertyDescriptor(fieldNameOptional.get());
+        final String rawPropertyValue = componentNode.getRawPropertyValue(descriptor);
+        return rawPropertyValue != null;
+    }
+
+    /**
      * Determines whether or not the given Process Group has a Connection whose source is the given Processor and that contains the given relationship
      *
      * @param processGroup the process group
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/test/java/org/apache/nifi/registry/flow/mapping/TestNiFiRegistryFlowMapper.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/test/java/org/apache/nifi/registry/flow/mapping/TestNiFiRegistryFlowMapper.java
new file mode 100644
index 0000000..dcc33cc
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/test/java/org/apache/nifi/registry/flow/mapping/TestNiFiRegistryFlowMapper.java
@@ -0,0 +1,221 @@
+/*
+ * 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.registry.flow.mapping;
+
+import org.apache.nifi.attribute.expression.language.VariableImpact;
+import org.apache.nifi.bundle.BundleCoordinate;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.Validator;
+import org.apache.nifi.connectable.Position;
+import org.apache.nifi.controller.ControllerService;
+import org.apache.nifi.controller.ProcessorNode;
+import org.apache.nifi.controller.PropertyConfiguration;
+import org.apache.nifi.controller.ScheduledState;
+import org.apache.nifi.controller.service.ControllerServiceNode;
+import org.apache.nifi.controller.service.ControllerServiceProvider;
+import org.apache.nifi.flow.VersionedProcessor;
+import org.apache.nifi.logging.LogLevel;
+import org.apache.nifi.nar.ExtensionManager;
+import org.apache.nifi.parameter.ExpressionLanguageAgnosticParameterParser;
+import org.apache.nifi.parameter.ParameterReference;
+import org.apache.nifi.parameter.ParameterTokenList;
+import org.apache.nifi.scheduling.ExecutionNode;
+import org.apache.nifi.scheduling.SchedulingStrategy;
+import org.junit.Test;
+import org.mockito.stubbing.Answer;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.UUID;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public class TestNiFiRegistryFlowMapper {
+    private static final SensitiveValueEncryptor ENCRYPTOR = value -> new StringBuilder(value).reverse().toString();
+
+    @Test
+    public void testMappingProcessorWithSensitiveValuesGivesNullValue() {
+        final ExtensionManager extensionManager = mock(ExtensionManager.class);
+        final FlowMappingOptions mappingOptions = FlowMappingOptions.DEFAULT_OPTIONS;
+        final NiFiRegistryFlowMapper mapper = new NiFiRegistryFlowMapper(extensionManager, mappingOptions);
+
+        final ControllerServiceProvider serviceProvider = mock(ControllerServiceProvider.class);
+        final Map<String, String> properties = new HashMap<>();
+        properties.put("Property A", "A");
+        properties.put("Sensitive Property B", "B");
+
+        final ProcessorNode procNode = createProcessorNode(properties);
+        final VersionedProcessor versionedProcessor = mapper.mapProcessor(procNode, serviceProvider, Collections.emptySet(), Collections.emptyMap());
+        assertNotNull(versionedProcessor);
+
+        final Map<String, String> versionedProperties = versionedProcessor.getProperties();
+        assertEquals("A", versionedProperties.get("Property A"));
+        assertNull(versionedProperties.get("Sensitive Property B"));
+    }
+
+
+
+    @Test
+    public void testMappingProcessorWithSensitiveValuesLeavesSensitiveParameterReference() {
+        final ExtensionManager extensionManager = mock(ExtensionManager.class);
+        final FlowMappingOptions mappingOptions = FlowMappingOptions.DEFAULT_OPTIONS;
+        final NiFiRegistryFlowMapper mapper = new NiFiRegistryFlowMapper(extensionManager, mappingOptions);
+
+        final ControllerServiceProvider serviceProvider = mock(ControllerServiceProvider.class);
+        final Map<String, String> properties = new HashMap<>();
+        properties.put("Property A", "A");
+        properties.put("Sensitive Property B", "#{Param B}");
+
+        final ProcessorNode procNode = createProcessorNode(properties);
+        final VersionedProcessor versionedProcessor = mapper.mapProcessor(procNode, serviceProvider, Collections.emptySet(), Collections.emptyMap());
+        assertNotNull(versionedProcessor);
+
+        final Map<String, String> versionedProperties = versionedProcessor.getProperties();
+        assertEquals("A", versionedProperties.get("Property A"));
+        assertEquals("#{Param B}", versionedProperties.get("Sensitive Property B"));
+    }
+
+    @Test
+    public void testMappingProcessorWithSensitiveValuesProvidesEncryptedValue() {
+        final ExtensionManager extensionManager = mock(ExtensionManager.class);
+        final FlowMappingOptions mappingOptions = new FlowMappingOptions.Builder()
+            .stateLookup(VersionedComponentStateLookup.IDENTITY_LOOKUP)
+            .componentIdLookup(ComponentIdLookup.USE_COMPONENT_ID)
+            .mapSensitiveConfiguration(true)
+            .sensitiveValueEncryptor(ENCRYPTOR)
+            .build();
+
+        final NiFiRegistryFlowMapper mapper = new NiFiRegistryFlowMapper(extensionManager, mappingOptions);
+
+        final ControllerServiceProvider serviceProvider = mock(ControllerServiceProvider.class);
+        final Map<String, String> properties = new HashMap<>();
+        properties.put("Property A", "A");
+        properties.put("Sensitive Property B", "A B C");
+
+        final ProcessorNode procNode = createProcessorNode(properties);
+        final VersionedProcessor versionedProcessor = mapper.mapProcessor(procNode, serviceProvider, Collections.emptySet(), Collections.emptyMap());
+        assertNotNull(versionedProcessor);
+
+        final Map<String, String> versionedProperties = versionedProcessor.getProperties();
+        assertEquals("A", versionedProperties.get("Property A"));
+        assertEquals("enc{C B A}", versionedProperties.get("Sensitive Property B"));
+    }
+
+    @Test
+    public void testControllerServicesMapsToVersionedId() {
+        testControllerServicesMapsToProperId(true);
+    }
+
+    @Test
+    public void testControllerServicesMapsToInstanceId() {
+        testControllerServicesMapsToProperId(false);
+    }
+
+
+    private void testControllerServicesMapsToProperId(final boolean useVersionedId) {
+        final ExtensionManager extensionManager = mock(ExtensionManager.class);
+        final FlowMappingOptions mappingOptions = new FlowMappingOptions.Builder()
+            .stateLookup(VersionedComponentStateLookup.IDENTITY_LOOKUP)
+            .componentIdLookup(ComponentIdLookup.VERSIONED_OR_GENERATE)
+            .mapControllerServiceReferencesToVersionedId(useVersionedId)
+            .build();
+
+        final NiFiRegistryFlowMapper mapper = new NiFiRegistryFlowMapper(extensionManager, mappingOptions);
+
+        final ControllerServiceNode mockServiceNode = mock(ControllerServiceNode.class);
+        when(mockServiceNode.getIdentifier()).thenReturn("1234");
+        when(mockServiceNode.getVersionedComponentId()).thenReturn(Optional.of("versioned"));
+
+        final ControllerServiceProvider serviceProvider = mock(ControllerServiceProvider.class);
+        when(serviceProvider.getControllerServiceNode("1234")).thenReturn(mockServiceNode);
+        final Map<String, String> properties = new HashMap<>();
+        properties.put("Service A", "1234");
+
+        final ProcessorNode procNode = createProcessorNode(properties);
+        final VersionedProcessor versionedProcessor = mapper.mapProcessor(procNode, serviceProvider, Collections.emptySet(), Collections.emptyMap());
+        assertNotNull(versionedProcessor);
+
+        final Map<String, String> versionedProperties = versionedProcessor.getProperties();
+        final String idReferenced = versionedProperties.get("Service A");
+        if (useVersionedId) {
+            assertEquals("versioned", idReferenced);
+        } else {
+            assertEquals("1234", idReferenced);
+        }
+    }
+
+
+    private ProcessorNode createProcessorNode(final Map<String, String> properties) {
+        final ExpressionLanguageAgnosticParameterParser parameterParser = new ExpressionLanguageAgnosticParameterParser();
+
+        final Map<PropertyDescriptor, PropertyConfiguration> propertyMap = new HashMap<>();
+        final Map<String, PropertyDescriptor> propertyDescriptors = new HashMap<>();
+
+        for (final Map.Entry<String, String> property : properties.entrySet()) {
+            final String propName = property.getKey();
+            final String value = property.getValue();
+
+            final PropertyDescriptor descriptor = new PropertyDescriptor.Builder()
+                .name(propName)
+                .sensitive(propName.toLowerCase().contains("sensitive"))
+                .identifiesControllerService(propName.toLowerCase().contains("service ") ? ControllerService.class : null)
+                .addValidator(Validator.VALID)
+                .build();
+
+            final ParameterTokenList tokenList = parameterParser.parseTokens(value);
+            final List<ParameterReference> referenceList = tokenList.toReferenceList();
+            final PropertyConfiguration configuration = new PropertyConfiguration(value, tokenList, referenceList, VariableImpact.NEVER_IMPACTED);
+            propertyMap.put(descriptor, configuration);
+            propertyDescriptors.put(propName, descriptor);
+        }
+
+        final ProcessorNode procNode = mock(ProcessorNode.class);
+        when(procNode.getProperties()).thenReturn(propertyMap);
+        when(procNode.getProperty(any(PropertyDescriptor.class))).thenAnswer((Answer<PropertyConfiguration>) invocation -> {
+            final PropertyDescriptor descriptor = invocation.getArgument(0, PropertyDescriptor.class);
+            return propertyMap.get(descriptor);
+        });
+
+        when(procNode.getRawPropertyValue(any(PropertyDescriptor.class))).thenAnswer((Answer<String>) invocation -> {
+            final PropertyDescriptor descriptor = invocation.getArgument(0, PropertyDescriptor.class);
+            return properties.get(descriptor.getName());
+        });
+
+        final String id = UUID.randomUUID().toString();
+        when(procNode.getIdentifier()).thenReturn(id);
+
+        when(procNode.getBulletinLevel()).thenReturn(LogLevel.WARN);
+        when(procNode.getExecutionNode()).thenReturn(ExecutionNode.ALL);
+        when(procNode.getPropertyDescriptors()).thenReturn(new ArrayList<>(propertyDescriptors.values()));
+        when(procNode.getBundleCoordinate()).thenReturn(new BundleCoordinate("group", "id", "version"));
+        when(procNode.getPosition()).thenReturn(new Position(0D, 0D));
+        when(procNode.getSchedulingStrategy()).thenReturn(SchedulingStrategy.TIMER_DRIVEN);
+        when(procNode.getPhysicalScheduledState()).thenReturn(ScheduledState.STOPPED);
+
+        return procNode;
+    }
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/cluster/protocol/DataFlow.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/cluster/protocol/DataFlow.java
index 734cc3c..854cc79 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/cluster/protocol/DataFlow.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/cluster/protocol/DataFlow.java
@@ -16,6 +16,7 @@
  */
 package org.apache.nifi.cluster.protocol;
 
+import org.apache.nifi.controller.flow.VersionedDataflow;
 import org.w3c.dom.Document;
 
 import java.util.Set;
@@ -30,6 +31,11 @@ public interface DataFlow {
     Document getFlowDocument();
 
     /**
+     * @return the VersionedDataflow represented by the bytes
+     */
+    VersionedDataflow getVersionedDataflow();
+
+    /**
      * @return the raw byte array of the snippets
      */
     byte[] getSnippets();
@@ -45,4 +51,8 @@ public interface DataFlow {
      */
     Set<String> getMissingComponents();
 
+    /**
+     * @return <code>true</code> if the contents are empty or are made up of XML, <code>false</code> if the contents are JSON
+     */
+    boolean isXml();
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractComponentNode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractComponentNode.java
index 1414754..24be80e 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractComponentNode.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractComponentNode.java
@@ -462,20 +462,21 @@ public abstract class AbstractComponentNode implements ComponentNode {
         final PropertyConfiguration oldConfiguration = properties.put(descriptor, propertyConfiguration);
         final String effectiveValue = propertyConfiguration.getEffectiveValue(getParameterContext());
 
-        if (!propertyConfiguration.equals(oldConfiguration)) {
-            if (descriptor.getControllerServiceDefinition() != null) {
-                if (oldConfiguration != null) {
-                    final String oldEffectiveValue = oldConfiguration.getEffectiveValue(getParameterContext());
-                    final ControllerServiceNode oldNode = serviceProvider.getControllerServiceNode(oldEffectiveValue);
-                    if (oldNode != null) {
-                        oldNode.removeReference(this, descriptor);
-                    }
+        // If the property references a Controller Service, we need to register this component & property descriptor as a reference.
+        // If it previously referenced a Controller Service, we need to also remove that reference.
+        // It is okay if the new & old values are the same - we just unregister the component/descriptor and re-register it.
+        if (descriptor.getControllerServiceDefinition() != null) {
+            if (oldConfiguration != null) {
+                final String oldEffectiveValue = oldConfiguration.getEffectiveValue(getParameterContext());
+                final ControllerServiceNode oldNode = serviceProvider.getControllerServiceNode(oldEffectiveValue);
+                if (oldNode != null) {
+                    oldNode.removeReference(this, descriptor);
                 }
+            }
 
-                final ControllerServiceNode newNode = serviceProvider.getControllerServiceNode(effectiveValue);
-                if (newNode != null) {
-                    newNode.addReference(this, descriptor);
-                }
+            final ControllerServiceNode newNode = serviceProvider.getControllerServiceNode(effectiveValue);
+            if (newNode != null) {
+                newNode.addReference(this, descriptor);
             }
         }
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ProcessorNode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ProcessorNode.java
index 61c6622..506c898 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ProcessorNode.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ProcessorNode.java
@@ -174,7 +174,7 @@ public abstract class ProcessorNode extends AbstractComponentNode implements Con
      * states such as STOPPING and STARTING.
      *
      * @return the physical state of this processor [DISABLED, STOPPED, RUNNING,
-     *         STARTIING, STOPIING]
+     *         STARTING, STOPPING]
      */
     public ScheduledState getPhysicalScheduledState() {
         return this.scheduledState.get();
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/flow/FlowManager.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/flow/FlowManager.java
index 250189c..5f33081 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/flow/FlowManager.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/flow/FlowManager.java
@@ -33,7 +33,6 @@ import org.apache.nifi.parameter.Parameter;
 import org.apache.nifi.parameter.ParameterContext;
 import org.apache.nifi.parameter.ParameterContextManager;
 import org.apache.nifi.web.api.dto.FlowSnippetDTO;
-import org.apache.nifi.web.api.entity.ParameterContextReferenceEntity;
 
 import java.net.URL;
 import java.util.Collection;
@@ -41,6 +40,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Optional;
 import java.util.Set;
+import java.util.function.Predicate;
 
 public interface FlowManager {
     String ROOT_GROUP_ID_ALIAS = "root";
@@ -181,6 +181,8 @@ public interface FlowManager {
 
     void onProcessorRemoved(ProcessorNode processor);
 
+    Set<ProcessorNode> findAllProcessors(Predicate<ProcessorNode> processorNode);
+
 
     /**
      * <p>
@@ -339,12 +341,13 @@ public interface FlowManager {
      * @param id                The unique id
      * @param name              The ParameterContext name
      * @param parameters        The Parameters
-     * @param parameterContexts Optional inherited ParameterContexts
+     * @param inheritedContextIds The identifiers of any Parameter Contexts that the newly created Parameter Context should inherit from. The order of the identifiers in the List determines the
+     * order in which parameters with conflicting names are resolved. I.e., the Parameter Context whose ID comes first in the List is preferred.
      * @return The created ParameterContext
      * @throws IllegalStateException If <code>parameterContexts</code> is not empty and this method is called without being wrapped
      * by {@link FlowManager#withParameterContextResolution(Runnable)}
      */
-    ParameterContext createParameterContext(String id, String name, Map<String, Parameter> parameters, List<ParameterContextReferenceEntity> parameterContexts);
+    ParameterContext createParameterContext(String id, String name, Map<String, Parameter> parameters, List<String> inheritedContextIds);
 
     /**
      * Performs the given ParameterContext-related action, and then resolves all inherited ParameterContext references.
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/flow/VersionedDataflow.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/flow/VersionedDataflow.java
new file mode 100644
index 0000000..b7e4a03
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/flow/VersionedDataflow.java
@@ -0,0 +1,101 @@
+/*
+ * 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.flow;
+
+import org.apache.nifi.flow.VersionedControllerService;
+import org.apache.nifi.flow.VersionedProcessGroup;
+import org.apache.nifi.flow.VersionedReportingTask;
+import org.apache.nifi.registry.flow.VersionedParameterContext;
+
+import java.util.List;
+import java.util.Set;
+
+public class VersionedDataflow {
+    private VersionedFlowEncodingVersion encodingVersion;
+    private int maxTimerDrivenThreadCount;
+    private List<VersionedRegistry> registries;
+    private List<VersionedParameterContext> parameterContexts;
+    private List<VersionedControllerService> controllerServices;
+    private List<VersionedReportingTask> reportingTasks;
+    private Set<VersionedTemplate> templates;
+    private VersionedProcessGroup rootGroup;
+
+    public VersionedFlowEncodingVersion getEncodingVersion() {
+        return encodingVersion;
+    }
+
+    public void setEncodingVersion(final VersionedFlowEncodingVersion encodingVersion) {
+        this.encodingVersion = encodingVersion;
+    }
+
+    public int getMaxTimerDrivenThreadCount() {
+        return maxTimerDrivenThreadCount;
+    }
+
+    public void setMaxTimerDrivenThreadCount(final int maxTimerDrivenThreadCount) {
+        this.maxTimerDrivenThreadCount = maxTimerDrivenThreadCount;
+    }
+
+    public List<VersionedRegistry> getRegistries() {
+        return registries;
+    }
+
+    public void setRegistries(final List<VersionedRegistry> registries) {
+        this.registries = registries;
+    }
+
+    public List<VersionedParameterContext> getParameterContexts() {
+        return parameterContexts;
+    }
+
+    public void setParameterContexts(final List<VersionedParameterContext> parameterContexts) {
+        this.parameterContexts = parameterContexts;
+    }
+
+    public List<VersionedControllerService> getControllerServices() {
+        return controllerServices;
+    }
+
+    public void setControllerServices(final List<VersionedControllerService> controllerServices) {
+        this.controllerServices = controllerServices;
+    }
+
+    public List<VersionedReportingTask> getReportingTasks() {
+        return reportingTasks;
+    }
+
+    public void setReportingTasks(final List<VersionedReportingTask> reportingTasks) {
+        this.reportingTasks = reportingTasks;
+    }
+
+    public VersionedProcessGroup getRootGroup() {
+        return rootGroup;
+    }
+
+    public void setRootGroup(final VersionedProcessGroup rootGroup) {
+        this.rootGroup = rootGroup;
+    }
+
+    public Set<VersionedTemplate> getTemplates() {
+        return templates;
+    }
+
+    public void setTemplates(final Set<VersionedTemplate> templates) {
+        this.templates = templates;
+    }
+}
diff --git a/nifi-api/src/main/java/org/apache/nifi/flow/ComponentType.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/flow/VersionedFlowEncodingVersion.java
similarity index 54%
copy from nifi-api/src/main/java/org/apache/nifi/flow/ComponentType.java
copy to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/flow/VersionedFlowEncodingVersion.java
index 954f164..1f48322 100644
--- a/nifi-api/src/main/java/org/apache/nifi/flow/ComponentType.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/flow/VersionedFlowEncodingVersion.java
@@ -15,35 +15,33 @@
  * limitations under the License.
  */
 
-package org.apache.nifi.flow;
+package org.apache.nifi.controller.flow;
 
-public enum ComponentType {
+public class VersionedFlowEncodingVersion {
+    private int majorVersion;
+    private int minorVersion;
 
-    CONNECTION("Connection"),
-    PROCESSOR("Processor"),
-    PROCESS_GROUP("Process Group"),
-    REMOTE_PROCESS_GROUP("Remote Process Group"),
-    INPUT_PORT("Input Port"),
-    OUTPUT_PORT("Output Port"),
-    REMOTE_INPUT_PORT("Remote Input Port"),
-    REMOTE_OUTPUT_PORT("Remote Output Port"),
-    FUNNEL("Funnel"),
-    LABEL("Label"),
-    CONTROLLER_SERVICE("Controller Service");
+    public VersionedFlowEncodingVersion() {
+    }
 
+    public VersionedFlowEncodingVersion(final int majorVersion, final int minorVersion) {
+        setMajorVersion(majorVersion);
+        setMinorVersion(minorVersion);
+    }
 
-    private final String typeName;
+    public int getMajorVersion() {
+        return majorVersion;
+    }
 
-    private ComponentType(final String typeName) {
-        this.typeName = typeName;
+    public void setMajorVersion(final int majorVersion) {
+        this.majorVersion = majorVersion;
     }
 
-    public String getTypeName() {
-        return typeName;
+    public int getMinorVersion() {
+        return minorVersion;
     }
 
-    @Override
-    public String toString() {
-        return typeName;
+    public void setMinorVersion(final int minorVersion) {
+        this.minorVersion = minorVersion;
     }
 }
diff --git a/nifi-registry/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/flow/VersionedParameterContext.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/flow/VersionedRegistry.java
similarity index 52%
copy from nifi-registry/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/flow/VersionedParameterContext.java
copy to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/flow/VersionedRegistry.java
index 9d5ab69..2e36c90 100644
--- a/nifi-registry/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/flow/VersionedParameterContext.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/flow/VersionedRegistry.java
@@ -14,53 +14,50 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.nifi.registry.flow;
 
-import io.swagger.annotations.ApiModelProperty;
-
-import java.util.List;
-import java.util.Set;
+package org.apache.nifi.controller.flow;
 
-public class VersionedParameterContext {
+import io.swagger.annotations.ApiModelProperty;
 
+public class VersionedRegistry {
+    private String id;
     private String name;
+    private String url;
     private String description;
-    private Set<VersionedParameter> parameters;
-    private List<String> inheritedParameterContexts;
 
-    @ApiModelProperty("The name of the context")
-    public String getName() {
-        return name;
+    @ApiModelProperty("The ID of the registry")
+    public String getId() {
+        return id;
     }
 
-    public void setName(String name) {
-        this.name = name;
+    public void setId(final String id) {
+        this.id = id;
     }
 
-    @ApiModelProperty("The description of the parameter context")
-    public String getDescription() {
-        return description;
+    @ApiModelProperty("The name of the registry")
+    public String getName() {
+        return name;
     }
 
-    public void setDescription(String description) {
-        this.description = description;
+    public void setName(final String name) {
+        this.name = name;
     }
 
-    @ApiModelProperty("The parameters in the context")
-    public Set<VersionedParameter> getParameters() {
-        return parameters;
+    @ApiModelProperty("The URL for interacting with the registry")
+    public String getUrl() {
+        return url;
     }
 
-    public void setParameters(Set<VersionedParameter> parameters) {
-        this.parameters = parameters;
+    public void setUrl(final String url) {
+        this.url = url;
     }
 
-    @ApiModelProperty("The names of additional parameter contexts from which to inherit parameters")
-    public List<String> getInheritedParameterContexts() {
-        return inheritedParameterContexts;
+    @ApiModelProperty("The description of the registry")
+    public String getDescription() {
+        return description;
     }
 
-    public void setInheritedParameterContexts(List<String> parameterContextNames) {
-        this.inheritedParameterContexts = parameterContextNames;
+    public void setDescription(final String description) {
+        this.description = description;
     }
 }
diff --git a/nifi-registry/nifi-registry-core/nifi-registry-flow-diff/src/main/java/org/apache/nifi/registry/flow/diff/StandardComparableDataFlow.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/flow/VersionedTemplate.java
similarity index 60%
copy from nifi-registry/nifi-registry-core/nifi-registry-flow-diff/src/main/java/org/apache/nifi/registry/flow/diff/StandardComparableDataFlow.java
copy to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/flow/VersionedTemplate.java
index 5c3965a..d9f2129 100644
--- a/nifi-registry/nifi-registry-core/nifi-registry-flow-diff/src/main/java/org/apache/nifi/registry/flow/diff/StandardComparableDataFlow.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/flow/VersionedTemplate.java
@@ -15,27 +15,25 @@
  * limitations under the License.
  */
 
-package org.apache.nifi.registry.flow.diff;
+package org.apache.nifi.controller.flow;
 
-import org.apache.nifi.flow.VersionedProcessGroup;
+import org.apache.nifi.flow.ComponentType;
+import org.apache.nifi.flow.VersionedComponent;
+import org.apache.nifi.web.api.dto.TemplateDTO;
 
-public class StandardComparableDataFlow implements ComparableDataFlow {
-    private final String name;
-    private final VersionedProcessGroup contents;
-
-    public StandardComparableDataFlow(final String name, final VersionedProcessGroup contents) {
-        this.name = name;
-        this.contents = contents;
-    }
+public class VersionedTemplate extends VersionedComponent {
+    private TemplateDTO templateDto;
 
     @Override
-    public String getName() {
-        return name;
+    public ComponentType getComponentType() {
+        return ComponentType.TEMPLATE;
     }
 
-    @Override
-    public VersionedProcessGroup getContents() {
-        return contents;
+    public TemplateDTO getTemplateDto() {
+        return templateDto;
     }
 
+    public void setTemplateDto(final TemplateDTO templateDto) {
+        this.templateDto = templateDto;
+    }
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/AbstractComponentScheduler.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/AbstractComponentScheduler.java
new file mode 100644
index 0000000..3add0ac
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/AbstractComponentScheduler.java
@@ -0,0 +1,72 @@
+/*
+ * 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.groups;
+
+import org.apache.nifi.connectable.Connectable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Queue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicLong;
+
+public abstract class AbstractComponentScheduler implements ComponentScheduler {
+    private static final Logger logger = LoggerFactory.getLogger(AbstractComponentScheduler.class);
+
+    private final AtomicLong pauseCount = new AtomicLong(0L);
+    private final Queue<Connectable> toStart = new LinkedBlockingQueue<>();
+
+    @Override
+    public void pause() {
+        final long count = pauseCount.incrementAndGet();
+        logger.debug("{} paused; count = {}", this, count);
+    }
+
+    @Override
+    public void resume() {
+        final long updatedCount = pauseCount.decrementAndGet();
+        logger.debug("{} resumed; count = {}", this, updatedCount);
+
+        if (updatedCount > 0) {
+            return;
+        }
+
+        Connectable connectable;
+        while ((connectable = toStart.poll()) != null) {
+            logger.debug("{} starting {}", this, connectable);
+            startNow(connectable);
+        }
+    }
+
+    private boolean isPaused() {
+        return pauseCount.get() > 0;
+    }
+
+    @Override
+    public void startComponent(final Connectable component) {
+        if (isPaused()) {
+            logger.debug("{} called to start {} but paused so will queue it for start later", this, component);
+            toStart.offer(component);
+        } else {
+            logger.debug("{} starting {} now", this, component);
+            startNow(component);
+        }
+    }
+
+    protected abstract void startNow(Connectable component);
+}
diff --git a/nifi-registry/nifi-registry-core/nifi-registry-flow-diff/src/main/java/org/apache/nifi/registry/flow/diff/ComparableDataFlow.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/BundleUpdateStrategy.java
similarity index 72%
copy from nifi-registry/nifi-registry-core/nifi-registry-flow-diff/src/main/java/org/apache/nifi/registry/flow/diff/ComparableDataFlow.java
copy to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/BundleUpdateStrategy.java
index a9ceca5..7a7ca0e 100644
--- a/nifi-registry/nifi-registry-core/nifi-registry-flow-diff/src/main/java/org/apache/nifi/registry/flow/diff/ComparableDataFlow.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/BundleUpdateStrategy.java
@@ -15,12 +15,18 @@
  * limitations under the License.
  */
 
-package org.apache.nifi.registry.flow.diff;
+package org.apache.nifi.groups;
 
-import org.apache.nifi.flow.VersionedProcessGroup;
+public enum BundleUpdateStrategy {
 
-public interface ComparableDataFlow {
-    String getName();
+    /**
+     * If a proposed flow has a different bundle than the current flow, the bundle is not updated
+     */
+    IGNORE_BUNDLE,
 
-    VersionedProcessGroup getContents();
+    USE_SPECIFIED_OR_COMPATIBLE_OR_GHOST,
+
+    USE_SPECIFIED_OR_GHOST,
+
+    USE_SPECIFIED_OR_FAIL;
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/cluster/protocol/DataFlow.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/ComponentIdGenerator.java
similarity index 55%
copy from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/cluster/protocol/DataFlow.java
copy to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/ComponentIdGenerator.java
index 734cc3c..0564234 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/cluster/protocol/DataFlow.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/ComponentIdGenerator.java
@@ -14,35 +14,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.nifi.cluster.protocol;
 
-import org.w3c.dom.Document;
-
-import java.util.Set;
-
-public interface DataFlow {
+package org.apache.nifi.groups;
 
+public interface ComponentIdGenerator {
     /**
-     * @return the raw byte array of the flow
+     * Generates a UUID based on the proposed component identifier and the UUID of the Process Group into which the component
+     * is going to be placed
+     *
+     * @param proposedId the proposed id, which is not guaranteed to be unique
+     * @param instanceId the instance ID of the Versioned component, or null if no instance ID is set
+     * @param destinationGroupId the UUID of the Process Group that the component is to be added to
+     * @return a UUID for the component
      */
-    byte[] getFlow();
-
-    Document getFlowDocument();
-
-    /**
-     * @return the raw byte array of the snippets
-     */
-    byte[] getSnippets();
-
-    /**
-     * @return the raw byte array of the Authorizer's fingerprint,
-     *              null when not using a ManagedAuthorizer
-     */
-    byte[] getAuthorizerFingerprint();
-
-    /**
-     * @return the component ids of components that were created as a missing ghost component
-     */
-    Set<String> getMissingComponents();
-
+    String generateUuid(String proposedId, String instanceId, String destinationGroupId);
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/mapping/InstantiatedVersionedComponent.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/ComponentScheduler.java
similarity index 62%
copy from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/mapping/InstantiatedVersionedComponent.java
copy to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/ComponentScheduler.java
index e655c16..bc6afcb 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/mapping/InstantiatedVersionedComponent.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/ComponentScheduler.java
@@ -15,16 +15,29 @@
  * limitations under the License.
  */
 
-package org.apache.nifi.registry.flow.mapping;
+package org.apache.nifi.groups;
 
-import javax.xml.bind.annotation.XmlTransient;
+import org.apache.nifi.connectable.Connectable;
 
-public interface InstantiatedVersionedComponent {
+public interface ComponentScheduler {
+    void startComponent(Connectable component);
 
-    // mark transient so fields are ignored when serializing all versioned component types
-    @XmlTransient
-    String getInstanceId();
+    void pause();
 
-    @XmlTransient
-    String getInstanceGroupId();
+    void resume();
+
+
+    ComponentScheduler NOP_SCHEDULER = new ComponentScheduler() {
+        @Override
+        public void startComponent(final Connectable component) {
+        }
+
+        @Override
+        public void pause() {
+        }
+
+        @Override
+        public void resume() {
+        }
+    };
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/GroupSynchronizationOptions.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/GroupSynchronizationOptions.java
new file mode 100644
index 0000000..661b986
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/GroupSynchronizationOptions.java
@@ -0,0 +1,220 @@
+/*
+ * 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.groups;
+
+public class GroupSynchronizationOptions {
+    private final ComponentIdGenerator componentIdGenerator;
+    private final ComponentScheduler componentScheduler;
+    private final PropertyDecryptor propertyDecryptor;
+    private final boolean ignoreLocalModifications;
+    private final boolean updateSettings;
+    private final boolean updateDescendantVersionedFlows;
+    private final boolean updateGroupVersionControlSnapshot;
+    private final boolean updateExistingVariables;
+    private final boolean updateRpgUrls;
+
+    private GroupSynchronizationOptions(final Builder builder) {
+        this.componentIdGenerator = builder.componentIdGenerator;
+        this.componentScheduler = builder.componentScheduler;
+        this.propertyDecryptor = builder.propertyDecryptor;
+        this.ignoreLocalModifications = builder.ignoreLocalModifications;
+        this.updateSettings = builder.updateSettings;
+        this.updateDescendantVersionedFlows = builder.updateDescendantVersionedFlows;
+        this.updateGroupVersionControlSnapshot = builder.updateGroupVersionControlSnapshot;
+        this.updateExistingVariables = builder.updateExistingVariables;
+        this.updateRpgUrls = builder.updateRpgUrls;
+    }
+
+    public ComponentIdGenerator getComponentIdGenerator() {
+        return componentIdGenerator;
+    }
+
+    public ComponentScheduler getComponentScheduler() {
+        return componentScheduler;
+    }
+
+    public boolean isIgnoreLocalModifications() {
+        return ignoreLocalModifications;
+    }
+
+    public boolean isUpdateSettings() {
+        return updateSettings;
+    }
+
+    public boolean isUpdateDescendantVersionedFlows() {
+        return updateDescendantVersionedFlows;
+    }
+
+    public boolean isUpdateGroupVersionControlSnapshot() {
+        return updateGroupVersionControlSnapshot;
+    }
+
+    public boolean isUpdateExistingVariables() {
+        return updateExistingVariables;
+    }
+
+    public boolean isUpdateRpgUrls() {
+        return updateRpgUrls;
+    }
+
+    public PropertyDecryptor getPropertyDecryptor() {
+        return propertyDecryptor;
+    }
+
+
+    public static class Builder {
+        private ComponentIdGenerator componentIdGenerator;
+        private ComponentScheduler componentScheduler;
+        private boolean ignoreLocalModifications = false;
+        private boolean updateSettings = true;
+        private boolean updateDescendantVersionedFlows = true;
+        private boolean updateGroupVersionControlSnapshot = true;
+        private boolean updateExistingVariables = false;
+        private boolean updateRpgUrls = false;
+        private PropertyDecryptor propertyDecryptor = value -> value;
+
+        /**
+         * Specifies the Component ID Generator to use for generating UUID's of components that are to be added to a ProcessGroup
+         * @param componentIdGenerator the ComponentIdGenerator to use
+         * @return the builder
+         */
+        public Builder componentIdGenerator(final ComponentIdGenerator componentIdGenerator) {
+            this.componentIdGenerator = componentIdGenerator;
+            return this;
+        }
+
+        /**
+         * Specifies the ComponentScheduler to use for starting connectable components
+         * @param componentScheduler the ComponentScheduler to use
+         * @return the builder
+         */
+        public Builder componentScheduler(final ComponentScheduler componentScheduler) {
+            this.componentScheduler = componentScheduler;
+            return this;
+        }
+
+        /**
+         * Specifies whether local modifications to a dataflow should prevent the flow from being updated
+         *
+         * @param ignore if <code>true</code>, the Process Group should be synchronized with the proposed VersionedProcessGroup even if it has local modifications.
+         * If <code>false</code>, an attempt to synchronize a Process Group with a proposed flow should fail
+         * @return the builder
+         */
+        public Builder ignoreLocalModifications(final boolean ignore) {
+            this.ignoreLocalModifications = ignore;
+            return this;
+        }
+
+        /**
+         * Specifies whether or not a Process Group's settings (e.g., name, position) should be updated
+         * @param updateSettings whether or not to update the Process Group's settings
+         * @return the builder
+         */
+        public Builder updateGroupSettings(final boolean updateSettings) {
+            this.updateSettings = updateSettings;
+            return this;
+        }
+
+        /**
+         * If a child Process Group is under version control, specifies whether or not the child should have its contents synchronized
+         * @param updateDescendantVersionedFlows <code>true</code> to synchronize child groups, <code>false</code> otherwise
+         * @return the builder
+         */
+        public Builder updateDescendantVersionedFlows(final boolean updateDescendantVersionedFlows) {
+            this.updateDescendantVersionedFlows = updateDescendantVersionedFlows;
+            return this;
+        }
+
+        /**
+         * When a Process Group is version controlled, it tracks whether or not there are any local modifications by comparing the current dataflow
+         * to a snapshot of what the Versioned Flow looks like. If this value is set to <code>true</code>, when the Process Group is synchronized
+         * with a VersionedProcessGroup, that VersionedProcessGroup will become the snapshot of what the Versioned Flow looks like. If <code>false</code>,
+         * the snapshot is not updated.
+         *
+         * @param updateGroupVersionControlSnapshot <code>true</code> to update the snapshot, <code>false</code> otherwise
+         * @return the builder
+         */
+        public Builder updateGroupVersionControlSnapshot(final boolean updateGroupVersionControlSnapshot) {
+            this.updateGroupVersionControlSnapshot = updateGroupVersionControlSnapshot;
+            return this;
+        }
+
+        /**
+         * Specifies whether or not existing variables in the Process Group's Variable Registry should be updated. If <code>false</code>, any Variable
+         * that exists in a Versioned Process Group will be added to the group's variable registry but existing variables will not be modified. If <code>true</code>,
+         * existing variables will also be updated
+         *
+         * @param updateExistingVariables whether or not to update existing variables
+         * @return the builder
+         */
+        public Builder updateExistingVariables(final boolean updateExistingVariables) {
+            this.updateExistingVariables = updateExistingVariables;
+            return this;
+        }
+
+        /**
+         * Specifies whether or not the URLs / "Target URIs" of a Remote Process Group that exists in both the proposed flow and the current flow
+         * should be updated to match that of the proposed flow
+         *
+         * @param updateRpgUrls whether or not to update the RPG URLs
+         * @return the builder
+         */
+        public Builder updateRpgUrls(final boolean updateRpgUrls) {
+            this.updateRpgUrls = updateRpgUrls;
+            return this;
+        }
+
+        /**
+         * Specifies the decryptor to use for sensitive properties
+         *
+         * @param decryptor the decryptor to use
+         * @return the builder
+         */
+        public Builder propertyDecryptor(final PropertyDecryptor decryptor) {
+            this.propertyDecryptor = decryptor;
+            return this;
+        }
+
+
+        public GroupSynchronizationOptions build() {
+            if (componentIdGenerator == null) {
+                throw new IllegalStateException("Must set Component ID Generator");
+            }
+            if (componentScheduler == null) {
+                throw new IllegalStateException("Must set Component Scheduler");
+            }
+
+            return new GroupSynchronizationOptions(this);
+        }
+
+        public static Builder from(final GroupSynchronizationOptions options) {
+            final Builder builder = new Builder();
+            builder.componentIdGenerator = options.getComponentIdGenerator();
+            builder.componentScheduler = options.getComponentScheduler();
+            builder.ignoreLocalModifications = options.isIgnoreLocalModifications();
+            builder.updateSettings = options.isUpdateSettings();
+            builder.updateDescendantVersionedFlows = options.isUpdateDescendantVersionedFlows();
+            builder.updateGroupVersionControlSnapshot = options.isUpdateGroupVersionControlSnapshot();
+            builder.updateExistingVariables = options.isUpdateExistingVariables();
+            builder.updateRpgUrls = options.isUpdateRpgUrls();
+            builder.propertyDecryptor = options.getPropertyDecryptor();
+
+            return builder;
+        }
+    }
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/ProcessGroup.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/ProcessGroup.java
index ee7200d..f642d36 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/ProcessGroup.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/ProcessGroup.java
@@ -41,6 +41,7 @@ import org.apache.nifi.registry.ComponentVariableRegistry;
 import org.apache.nifi.registry.flow.FlowRegistryClient;
 import org.apache.nifi.registry.flow.VersionControlInformation;
 import org.apache.nifi.registry.flow.VersionedFlowSnapshot;
+import org.apache.nifi.registry.flow.mapping.FlowMappingOptions;
 import org.apache.nifi.remote.RemoteGroupPort;
 
 import java.util.Collection;
@@ -882,6 +883,15 @@ public interface ProcessGroup extends ComponentAuthorizable, Positionable, Versi
     void updateFlow(VersionedFlowSnapshot proposedSnapshot, String componentIdSeed, boolean verifyNotDirty, boolean updateSettings, boolean updateDescendantVersionedFlows);
 
     /**
+     * Updates the Process Group to match the proposed flow
+     *
+     * @param proposedSnapshot the proposed flow
+     * @param synchronizationOptions options for how the synchronization should occur
+     * @param flowMappingOptions options for how to map the existing dataflow into Versioned components so that it can be compared to the proposed snapshot
+     */
+    void synchronizeFlow(VersionedFlowSnapshot proposedSnapshot, GroupSynchronizationOptions synchronizationOptions, FlowMappingOptions flowMappingOptions);
+
+    /**
      * Verifies a template with the specified name can be created.
      *
      * @param name name of the template
@@ -1207,7 +1217,7 @@ public interface ProcessGroup extends ComponentAuthorizable, Positionable, Versi
     void setDefaultBackPressureObjectThreshold(Long defaultBackPressureObjectThreshold);
 
     /**
-     * @returnthe default back pressure size threshold of this ProcessGroup
+     * @return the default back pressure size threshold of this ProcessGroup
      */
     String getDefaultBackPressureDataSizeThreshold();
 
diff --git a/nifi-api/src/main/java/org/apache/nifi/flow/ScheduledState.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/PropertyDecryptor.java
similarity index 83%
copy from nifi-api/src/main/java/org/apache/nifi/flow/ScheduledState.java
copy to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/PropertyDecryptor.java
index aaba9ae..b88bd68 100644
--- a/nifi-api/src/main/java/org/apache/nifi/flow/ScheduledState.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/PropertyDecryptor.java
@@ -14,9 +14,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.nifi.flow;
 
-public enum ScheduledState {
-    ENABLED,
-    DISABLED;
+package org.apache.nifi.groups;
+
+public interface PropertyDecryptor {
+    String decrypt(String value);
+
+    PropertyDecryptor NO_OP_DECRYPTOR = value -> value;
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/registry/flow/mapping/ComponentIdLookup.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/registry/flow/mapping/ComponentIdLookup.java
new file mode 100644
index 0000000..a8039ec
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/registry/flow/mapping/ComponentIdLookup.java
@@ -0,0 +1,55 @@
+/*
+ * 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.registry.flow.mapping;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Optional;
+import java.util.UUID;
+
+public interface ComponentIdLookup {
+
+    /**
+     * Given a component identifier and an optional Versioned Component ID, returns the identifier to use for the component
+     * @param currentVersionedId the current Versioned Component ID, or an empty optional if the component does not currently have a Versioned Component ID
+     * @param componentId the ID of the component
+     * @return the ID to use for mapping a component to a Versioned Component
+     */
+    String getComponentId(Optional<String> currentVersionedId, String componentId);
+
+
+    /**
+     * Uses the Versioned Component ID, if it is present, or else generates a new Versioned Component ID based on the Component ID
+     */
+    ComponentIdLookup VERSIONED_OR_GENERATE = new ComponentIdLookup() {
+        @Override
+        public String getComponentId(final Optional<String> currentVersionedId, final String componentId) {
+            if (currentVersionedId.isPresent()) {
+                return currentVersionedId.get();
+            }
+
+            return UUID.nameUUIDFromBytes(componentId.getBytes(StandardCharsets.UTF_8)).toString();
+        }
+    };
+
+
+
+    /**
+     * Always uses the Component ID
+     */
+    ComponentIdLookup USE_COMPONENT_ID = (versioned, componentId) -> componentId;
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/registry/flow/mapping/FlowMappingOptions.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/registry/flow/mapping/FlowMappingOptions.java
new file mode 100644
index 0000000..d5f9e19
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/registry/flow/mapping/FlowMappingOptions.java
@@ -0,0 +1,197 @@
+/*
+ * 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.registry.flow.mapping;
+
+import static java.util.Objects.requireNonNull;
+
+public class FlowMappingOptions {
+    private final SensitiveValueEncryptor encryptor;
+    private final VersionedComponentStateLookup stateLookup;
+    private final ComponentIdLookup componentIdLookup;
+    private final boolean mapPropertyDescriptors;
+    private final boolean mapSensitiveConfiguration;
+    private final boolean mapInstanceIds;
+    private final boolean mapControllerServiceReferencesToVersionedId;
+
+    private FlowMappingOptions(final Builder builder) {
+        encryptor = builder.encryptor;
+        stateLookup = builder.stateLookup;
+        componentIdLookup = builder.componentIdLookup;
+        mapPropertyDescriptors = builder.mapPropertyDescriptors;
+        mapSensitiveConfiguration = builder.mapSensitiveConfiguration;
+        mapInstanceIds = builder.mapInstanceId;
+        mapControllerServiceReferencesToVersionedId = builder.mapControllerServiceReferencesToVersionedId;
+    }
+
+    public SensitiveValueEncryptor getSensitiveValueEncryptor() {
+        return encryptor;
+    }
+
+    public VersionedComponentStateLookup getStateLookup() {
+        return stateLookup;
+    }
+
+    public ComponentIdLookup getComponentIdLookup() {
+        return componentIdLookup;
+    }
+
+    public boolean isMapPropertyDescriptors() {
+        return mapPropertyDescriptors;
+    }
+
+    public boolean isMapSensitiveConfiguration() {
+        return mapSensitiveConfiguration;
+    }
+
+    public boolean isMapInstanceIdentifiers() {
+        return mapInstanceIds;
+    }
+
+    public boolean isMapControllerServiceReferencesToVersionedId() {
+        return mapControllerServiceReferencesToVersionedId;
+    }
+
+    public static class Builder {
+        private SensitiveValueEncryptor encryptor;
+        private VersionedComponentStateLookup stateLookup;
+        private ComponentIdLookup componentIdLookup;
+        private boolean mapPropertyDescriptors;
+        private boolean mapSensitiveConfiguration;
+        private boolean mapInstanceId = false;
+        private boolean mapControllerServiceReferencesToVersionedId = true;
+
+        /**
+         * Sets the SensitiveValueEncryptor to use for encrypting sensitive values. This value must be set
+         * if {@link #mapSensitiveConfiguration(boolean) mapSensitiveConfiguration} is set to <code>true</code>.
+         *
+         * @param encryptor the PropertyEncryptor to use
+         * @return the builder
+         */
+        public Builder sensitiveValueEncryptor(final SensitiveValueEncryptor encryptor) {
+            this.encryptor = encryptor;
+            return this;
+        }
+
+        /**
+         * Sets the State Lookup to use. When a component is mapped to a Versioned Component, this is used to determine
+         * which ScheduledState should be assigned to the VersionedComponent
+         *
+         * @param stateLookup the State Lookup to use
+         * @return the builder
+         */
+        public Builder stateLookup(final VersionedComponentStateLookup stateLookup) {
+            this.stateLookup = stateLookup;
+            return this;
+        }
+
+        /**
+         * Sets the ComponentIdLookup to use. Given an existing component, the Component ID Lookup can be used to determine
+         * how the component's identifier and its (optional) versioned component identifier should be used to derive an identifier
+         * for the Versioned Component
+         *
+         * @param componentIdLookup the Component ID Lookup to use
+         * @return the builder
+         */
+        public Builder componentIdLookup(final ComponentIdLookup componentIdLookup) {
+            this.componentIdLookup = componentIdLookup;
+            return this;
+        }
+
+        /**
+         * Sets whether or not to map the component's Property Descriptors to the Versioned Component. If <code>false</code>, the Property Descriptors
+         * will not be set for components such as Processor, Controller Services, and Reporting Tasks.
+         *
+         * @param mapPropertyDescriptors whether or not to map property descriptors
+         * @return the builder
+         */
+        public Builder mapPropertyDescriptors(final boolean mapPropertyDescriptors) {
+            this.mapPropertyDescriptors = mapPropertyDescriptors;
+            return this;
+        }
+
+        /**
+         * Sets whether or not to map sensitive values. If <code>true</code>, the {@link #sensitiveValueEncryptor(SensitiveValueEncryptor)} must be set
+         *
+         * @param mapSensitiveConfiguration whether or not sensitive values should be mapped
+         * @return the builder
+         */
+        public Builder mapSensitiveConfiguration(final boolean mapSensitiveConfiguration) {
+            this.mapSensitiveConfiguration = mapSensitiveConfiguration;
+            return this;
+        }
+
+        /**
+         * Sets whether or not the Versioned Components' Instance Identifiers should be populated
+         *
+         * @param mapInstanceIdentifiers whether or not to map a component's identifier to the VersionedComponent's instanceId
+         * @return the builder
+         */
+        public Builder mapInstanceIdentifiers(final boolean mapInstanceIdentifiers) {
+            this.mapInstanceId = mapInstanceIdentifiers;
+            return this;
+        }
+
+        /**
+         * Specifies how Controller Service references should be mapped. If Processor A references Controller Service B, and this value is
+         * set to <code>true</code>, the VersionedProcessor will have a property that references the Versioned Component ID for the Controller Service.
+         * If set to <code>false</code>, the VersionedProcessor's property value will match that of the processor itself, mapping to the ID of the
+         * instantiated Controller Service.
+         *
+         * @param mapControllerServiceReferencesToVersionedId whether or not to map Controller Service References to hte Versioned Component ID
+         * @return the builder
+         */
+        public Builder mapControllerServiceReferencesToVersionedId(final boolean mapControllerServiceReferencesToVersionedId) {
+            this.mapControllerServiceReferencesToVersionedId = mapControllerServiceReferencesToVersionedId;
+            return this;
+        }
+
+        /**
+         * Creates a FlowMappingOptions object, or throws an Exception if not all required configuration has been provided
+         *
+         * @return the FlowMappingOptions
+         * @throws NullPointerException if the {@link #stateLookup(VersionedComponentStateLookup) StateLookup} is not set, the
+         * {@link #componentIdLookup(ComponentIdLookup) ComponentIdLookup} is not set, or if {@link #mapSensitiveConfiguration(boolean) mapSensitiveConfiguration}
+         * is set to true but the {@link #sensitiveValueEncryptor(SensitiveValueEncryptor) SensitiveValueEncryptor} has not been set
+         */
+        public FlowMappingOptions build() {
+            requireNonNull(stateLookup, "State Lookup must be set");
+            requireNonNull(componentIdLookup, "Component ID Lookup must be set");
+
+            if (mapSensitiveConfiguration) {
+                requireNonNull(encryptor, "Property Encryptor must be set when sensitive configuration is to be mapped");
+            }
+
+            return new FlowMappingOptions(this);
+        }
+    }
+
+    /**
+     * The Default Options are acceptable for sharing a dataflow externally (outside of a given nifi instance or cluster), such as sharing
+     * a dataflow to a NiFi Registry.
+     */
+    public static final FlowMappingOptions DEFAULT_OPTIONS = new Builder()
+        .sensitiveValueEncryptor(null)
+        .stateLookup(VersionedComponentStateLookup.ENABLED_OR_DISABLED)
+        .componentIdLookup(ComponentIdLookup.VERSIONED_OR_GENERATE)
+        .mapPropertyDescriptors(true)
+        .mapSensitiveConfiguration(false)
+        .mapInstanceIdentifiers(false)
+        .mapControllerServiceReferencesToVersionedId(true)
+        .build();
+
+}
diff --git a/nifi-api/src/main/java/org/apache/nifi/flow/ScheduledState.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/registry/flow/mapping/SensitiveValueEncryptor.java
similarity index 86%
copy from nifi-api/src/main/java/org/apache/nifi/flow/ScheduledState.java
copy to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/registry/flow/mapping/SensitiveValueEncryptor.java
index aaba9ae..ce43c46 100644
--- a/nifi-api/src/main/java/org/apache/nifi/flow/ScheduledState.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/registry/flow/mapping/SensitiveValueEncryptor.java
@@ -14,9 +14,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.nifi.flow;
 
-public enum ScheduledState {
-    ENABLED,
-    DISABLED;
+package org.apache.nifi.registry.flow.mapping;
+
+public interface SensitiveValueEncryptor {
+    String encrypt(String value);
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/registry/flow/mapping/VersionedComponentStateLookup.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/registry/flow/mapping/VersionedComponentStateLookup.java
new file mode 100644
index 0000000..a1ceccd
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/registry/flow/mapping/VersionedComponentStateLookup.java
@@ -0,0 +1,108 @@
+/*
+ * 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.registry.flow.mapping;
+
+import org.apache.nifi.connectable.Port;
+import org.apache.nifi.controller.ProcessorNode;
+import org.apache.nifi.controller.ReportingTaskNode;
+import org.apache.nifi.controller.service.ControllerServiceNode;
+import org.apache.nifi.controller.service.ControllerServiceState;
+import org.apache.nifi.flow.ScheduledState;
+
+public interface VersionedComponentStateLookup {
+    ScheduledState getState(ProcessorNode processorNode);
+
+    ScheduledState getState(Port port);
+
+    ScheduledState getState(ReportingTaskNode taskNode);
+
+    ScheduledState getState(ControllerServiceNode serviceNode);
+
+    /**
+     * Returns a Scheduled State of ENABLED or DISABLED for every component. No component will be mapped to RUNNING.
+     */
+    VersionedComponentStateLookup ENABLED_OR_DISABLED = new VersionedComponentStateLookup() {
+        @Override
+        public ScheduledState getState(final ProcessorNode processorNode) {
+            return processorNode.getScheduledState() == org.apache.nifi.controller.ScheduledState.DISABLED ? ScheduledState.DISABLED : ScheduledState.ENABLED;
+        }
+
+        @Override
+        public ScheduledState getState(final Port port) {
+            return port.getScheduledState() == org.apache.nifi.controller.ScheduledState.DISABLED ? ScheduledState.DISABLED : ScheduledState.ENABLED;
+        }
+
+        @Override
+        public ScheduledState getState(final ReportingTaskNode taskNode) {
+            return taskNode.getScheduledState() == org.apache.nifi.controller.ScheduledState.DISABLED ? ScheduledState.DISABLED : ScheduledState.ENABLED;
+        }
+
+        @Override
+        public ScheduledState getState(final ControllerServiceNode serviceNode) {
+            return (serviceNode.getState() == ControllerServiceState.DISABLED || serviceNode.getState() == ControllerServiceState.DISABLING) ? ScheduledState.DISABLED : ScheduledState.ENABLED;
+        }
+    };
+
+    /**
+     * Returns the Scheduled State according to whatever is currently set for the component
+     */
+    VersionedComponentStateLookup IDENTITY_LOOKUP = new VersionedComponentStateLookup() {
+        @Override
+        public ScheduledState getState(final ProcessorNode processorNode) {
+            return map(processorNode.getPhysicalScheduledState());
+        }
+
+        @Override
+        public ScheduledState getState(final Port port) {
+            return map(port.getScheduledState());
+        }
+
+        @Override
+        public ScheduledState getState(final ReportingTaskNode taskNode) {
+            return map(taskNode.getScheduledState());
+        }
+
+        @Override
+        public ScheduledState getState(final ControllerServiceNode serviceNode) {
+            switch (serviceNode.getState()) {
+                case ENABLED:
+                case ENABLING:
+                    return ScheduledState.ENABLED;
+                case DISABLED:
+                case DISABLING:
+                default:
+                    return ScheduledState.DISABLED;
+            }
+        }
+
+        private ScheduledState map(final org.apache.nifi.controller.ScheduledState componentState) {
+            switch (componentState) {
+                case DISABLED:
+                    return ScheduledState.DISABLED;
+                case RUNNING:
+                case STARTING:
+                    return ScheduledState.RUNNING;
+                case RUN_ONCE:
+                case STOPPED:
+                case STOPPING:
+                default:
+                    return ScheduledState.ENABLED;
+            }
+        }
+    };
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
index cbf9cbe..c166ba6 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
@@ -139,8 +139,11 @@ import org.apache.nifi.engine.FlowEngine;
 import org.apache.nifi.events.BulletinFactory;
 import org.apache.nifi.events.EventReporter;
 import org.apache.nifi.flow.Bundle;
+import org.apache.nifi.flow.VersionedConnection;
+import org.apache.nifi.flow.VersionedProcessGroup;
 import org.apache.nifi.flowfile.FlowFilePrioritizer;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.groups.BundleUpdateStrategy;
 import org.apache.nifi.groups.ProcessGroup;
 import org.apache.nifi.groups.RemoteProcessGroup;
 import org.apache.nifi.groups.StandardProcessGroup;
@@ -164,8 +167,6 @@ import org.apache.nifi.provenance.StandardProvenanceAuthorizableFactory;
 import org.apache.nifi.provenance.StandardProvenanceEventRecord;
 import org.apache.nifi.registry.VariableRegistry;
 import org.apache.nifi.registry.flow.FlowRegistryClient;
-import org.apache.nifi.flow.VersionedConnection;
-import org.apache.nifi.flow.VersionedProcessGroup;
 import org.apache.nifi.registry.variable.MutableVariableRegistry;
 import org.apache.nifi.remote.HttpRemoteSiteListener;
 import org.apache.nifi.remote.RemoteGroupPort;
@@ -304,8 +305,8 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
     private final Integer remoteInputHttpPort;
     private final Boolean isSiteToSiteSecure;
 
-    private final List<Connectable> startConnectablesAfterInitialization;
-    private final List<RemoteGroupPort> startRemoteGroupPortsAfterInitialization;
+    private final Set<Connectable> startConnectablesAfterInitialization;
+    private final Set<RemoteGroupPort> startRemoteGroupPortsAfterInitialization;
     private final LeaderElectionManager leaderElectionManager;
     private final ClusterCoordinator clusterCoordinator;
     private final FlowRegistryClient flowRegistryClient;
@@ -563,8 +564,8 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
         processScheduler.setSchedulingAgent(SchedulingStrategy.PRIMARY_NODE_ONLY, timerDrivenAgent);
         processScheduler.setSchedulingAgent(SchedulingStrategy.CRON_DRIVEN, quartzSchedulingAgent);
 
-        startConnectablesAfterInitialization = new ArrayList<>();
-        startRemoteGroupPortsAfterInitialization = new ArrayList<>();
+        startConnectablesAfterInitialization = new HashSet<>();
+        startRemoteGroupPortsAfterInitialization = new HashSet<>();
 
         final String gracefulShutdownSecondsVal = nifiProperties.getProperty(GRACEFUL_SHUTDOWN_PERIOD);
         long shutdownSecs;
@@ -1465,14 +1466,14 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
      * @throws MissingBundleException       if the proposed flow cannot be loaded by the
      *                                      controller because it contains a bundle that does not exist in the controller
      */
-    public void synchronize(final FlowSynchronizer synchronizer, final DataFlow dataFlow, final FlowService flowService)
+    public void synchronize(final FlowSynchronizer synchronizer, final DataFlow dataFlow, final FlowService flowService, final BundleUpdateStrategy bundleUpdateStrategy)
             throws FlowSerializationException, FlowSynchronizationException, UninheritableFlowException, MissingBundleException {
         writeLock.lock();
         try {
             LOG.debug("Synchronizing controller with proposed flow");
 
             try {
-                synchronizer.sync(this, dataFlow, encryptor, flowService);
+                synchronizer.sync(this, dataFlow, encryptor, flowService, bundleUpdateStrategy);
             } catch (final UninheritableFlowException ufe) {
                 final NodeIdentifier localNodeId = getNodeId();
                 if (localNodeId != null) {
@@ -1566,7 +1567,7 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
      * @throws IllegalStateException    if the FlowController does not know about
      *                                  the given process group
      */
-    void setRootGroup(final ProcessGroup group) {
+    public void setRootGroup(final ProcessGroup group) {
         if (requireNonNull(group).getParent() != null) {
             throw new IllegalArgumentException("A ProcessGroup that has a parent cannot be the Root Group");
         }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowService.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowService.java
index 49ae6d3..70eb8b0 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowService.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowService.java
@@ -52,17 +52,19 @@ import org.apache.nifi.connectable.Connection;
 import org.apache.nifi.controller.flow.FlowManager;
 import org.apache.nifi.controller.serialization.FlowSerializationException;
 import org.apache.nifi.controller.serialization.FlowSynchronizationException;
+import org.apache.nifi.controller.serialization.StandardFlowSynchronizer;
 import org.apache.nifi.controller.status.ProcessGroupStatus;
 import org.apache.nifi.encrypt.PropertyEncryptor;
 import org.apache.nifi.engine.FlowEngine;
 import org.apache.nifi.events.BulletinFactory;
+import org.apache.nifi.groups.BundleUpdateStrategy;
 import org.apache.nifi.groups.ProcessGroup;
 import org.apache.nifi.groups.RemoteProcessGroup;
 import org.apache.nifi.lifecycle.LifeCycleStartException;
 import org.apache.nifi.logging.LogLevel;
 import org.apache.nifi.nar.NarClassLoadersHolder;
 import org.apache.nifi.persistence.FlowConfigurationDAO;
-import org.apache.nifi.persistence.StandardXMLFlowConfigurationDAO;
+import org.apache.nifi.persistence.StandardFlowConfigurationDAO;
 import org.apache.nifi.persistence.TemplateDeserializer;
 import org.apache.nifi.reporting.Bulletin;
 import org.apache.nifi.reporting.EventAccess;
@@ -70,7 +72,6 @@ import org.apache.nifi.services.FlowService;
 import org.apache.nifi.stream.io.GZIPOutputStream;
 import org.apache.nifi.util.FormatUtils;
 import org.apache.nifi.util.NiFiProperties;
-import org.apache.nifi.util.file.FileUtils;
 import org.apache.nifi.web.api.dto.TemplateDTO;
 import org.apache.nifi.web.revision.RevisionManager;
 import org.apache.nifi.web.revision.RevisionSnapshot;
@@ -83,14 +84,10 @@ 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.net.InetSocketAddress;
 import java.nio.charset.StandardCharsets;
-import java.nio.file.Files;
 import java.nio.file.Path;
-import java.nio.file.Paths;
-import java.nio.file.StandardOpenOption;
 import java.util.ArrayList;
 import java.util.Calendar;
 import java.util.Collections;
@@ -107,7 +104,6 @@ import java.util.concurrent.atomic.AtomicReference;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 import java.util.stream.Collectors;
-import java.util.zip.GZIPInputStream;
 
 public class StandardFlowService implements FlowService, ProtocolHandler {
 
@@ -118,7 +114,6 @@ public class StandardFlowService implements FlowService, ProtocolHandler {
     private static final String NODE_UUID = "Node UUID";
 
     private final FlowController controller;
-    private final Path flowXml;
     private final FlowConfigurationDAO dao;
     private final int gracefulShutdownSeconds;
     private final boolean autoResumeState;
@@ -192,12 +187,11 @@ public class StandardFlowService implements FlowService, ProtocolHandler {
 
         this.nifiProperties = nifiProperties;
         this.controller = controller;
-        flowXml = Paths.get(nifiProperties.getProperty(NiFiProperties.FLOW_CONFIGURATION_FILE));
 
         gracefulShutdownSeconds = (int) FormatUtils.getTimeDuration(nifiProperties.getProperty(NiFiProperties.FLOW_CONTROLLER_GRACEFUL_SHUTDOWN_PERIOD), TimeUnit.SECONDS);
         autoResumeState = nifiProperties.getAutoResumeState();
 
-        dao = new StandardXMLFlowConfigurationDAO(flowXml, encryptor, nifiProperties, controller.getExtensionManager());
+        dao = new StandardFlowConfigurationDAO(encryptor, nifiProperties, controller.getExtensionManager());
         this.clusterCoordinator = clusterCoordinator;
         if (clusterCoordinator != null) {
             clusterCoordinator.setFlowService(this);
@@ -455,7 +449,7 @@ public class StandardFlowService implements FlowService, ProtocolHandler {
             // whole flow into the flow controller and applied any bundle upgrades
             writeLock.lock();
             try {
-                loadFromBytes(initialFlow, true);
+                loadFromBytes(initialFlow, true, BundleUpdateStrategy.USE_SPECIFIED_OR_COMPATIBLE_OR_GHOST);
             } finally {
                 writeLock.unlock();
             }
@@ -473,7 +467,7 @@ public class StandardFlowService implements FlowService, ProtocolHandler {
              * the response will be null and we should load the local dataflow
              * and heartbeat until a manager is located.
              */
-            final boolean localFlowEmpty = StandardFlowSynchronizer.isEmpty(proposedFlow);
+            final boolean localFlowEmpty = StandardFlowSynchronizer.isFlowEmpty(proposedFlow);
             final ConnectionResponse response = connect(true, localFlowEmpty, proposedFlow);
 
             // obtain write lock while we are updating the controller. We need to ensure that we don't
@@ -536,7 +530,7 @@ public class StandardFlowService implements FlowService, ProtocolHandler {
             writeLock.lock();
             try {
                 // operating in standalone mode, so load proposed flow and initialize the controller
-                loadFromBytes(dataFlow, true);
+                loadFromBytes(dataFlow, true, BundleUpdateStrategy.USE_SPECIFIED_OR_COMPATIBLE_OR_GHOST);
                 initializeController();
                 dao.save(controller, true);
             } finally {
@@ -779,7 +773,7 @@ public class StandardFlowService implements FlowService, ProtocolHandler {
     }
 
     // write lock must already be acquired
-    private void loadFromBytes(final DataFlow proposedFlow, final boolean allowEmptyFlow)
+    private void loadFromBytes(final DataFlow proposedFlow, final boolean allowEmptyFlow, final BundleUpdateStrategy bundleUpdateStrategy)
             throws IOException, FlowSerializationException, FlowSynchronizationException, UninheritableFlowException, MissingBundleException {
         logger.trace("Loading flow from bytes");
 
@@ -807,7 +801,7 @@ public class StandardFlowService implements FlowService, ProtocolHandler {
 
         // load the flow
         logger.debug("Loading proposed flow into FlowController");
-        dao.load(controller, actualProposedFlow, this);
+        dao.load(controller, actualProposedFlow, this, bundleUpdateStrategy);
 
         final ProcessGroup rootGroup = controller.getFlowManager().getRootGroup();
         if (rootGroup.isEmpty() && !allowEmptyFlow) {
@@ -1018,7 +1012,7 @@ public class StandardFlowService implements FlowService, ProtocolHandler {
             controller.setNodeId(nodeId);
 
             // load new controller state
-            loadFromBytes(dataFlow, true);
+            loadFromBytes(dataFlow, true, BundleUpdateStrategy.USE_SPECIFIED_OR_FAIL);
 
             // set node ID on controller before we start heartbeating because heartbeat needs node ID
             clusterCoordinator.setLocalNodeIdentifier(nodeId);
@@ -1071,14 +1065,7 @@ public class StandardFlowService implements FlowService, ProtocolHandler {
     public void copyCurrentFlow(final OutputStream os) throws IOException {
         readLock.lock();
         try {
-            if (!Files.exists(flowXml) || Files.size(flowXml) == 0) {
-                return;
-            }
-
-            try (final InputStream in = Files.newInputStream(flowXml, StandardOpenOption.READ);
-                    final InputStream gzipIn = new GZIPInputStream(in)) {
-                FileUtils.copy(gzipIn, os);
-            }
+            dao.load(os);
         } finally {
             readLock.unlock();
         }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowSynchronizer.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/XmlFlowSynchronizer.java
similarity index 99%
rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowSynchronizer.java
rename to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/XmlFlowSynchronizer.java
index be164da..ab15df7 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowSynchronizer.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/XmlFlowSynchronizer.java
@@ -57,6 +57,7 @@ import org.apache.nifi.controller.service.ControllerServiceState;
 import org.apache.nifi.encrypt.PropertyEncryptor;
 import org.apache.nifi.events.BulletinFactory;
 import org.apache.nifi.flowfile.FlowFilePrioritizer;
+import org.apache.nifi.groups.BundleUpdateStrategy;
 import org.apache.nifi.groups.FlowFileConcurrency;
 import org.apache.nifi.groups.FlowFileOutboundPolicy;
 import org.apache.nifi.groups.ProcessGroup;
@@ -137,31 +138,24 @@ import java.util.zip.GZIPInputStream;
 
 /**
  */
-public class StandardFlowSynchronizer implements FlowSynchronizer {
+public class XmlFlowSynchronizer implements FlowSynchronizer {
 
-    private static final Logger logger = LoggerFactory.getLogger(StandardFlowSynchronizer.class);
+    private static final Logger logger = LoggerFactory.getLogger(XmlFlowSynchronizer.class);
     private final PropertyEncryptor encryptor;
     private final boolean autoResumeState;
     private final NiFiProperties nifiProperties;
     private final ExtensionManager extensionManager;
 
-    public StandardFlowSynchronizer(final PropertyEncryptor encryptor, final NiFiProperties nifiProperties, final ExtensionManager extensionManager) {
+    public XmlFlowSynchronizer(final PropertyEncryptor encryptor, final NiFiProperties nifiProperties, final ExtensionManager extensionManager) {
         this.encryptor = encryptor;
         this.autoResumeState = nifiProperties.getAutoResumeState();
         this.nifiProperties = nifiProperties;
         this.extensionManager = extensionManager;
     }
 
-    public static boolean isEmpty(final DataFlow dataFlow) {
-        if (dataFlow == null || dataFlow.getFlow() == null || dataFlow.getFlow().length == 0) {
-            return true;
-        }
-
-        return isFlowEmpty(dataFlow.getFlowDocument());
-    }
 
     @Override
-    public void sync(final FlowController controller, final DataFlow proposedFlow, final PropertyEncryptor encryptor, final FlowService flowService)
+    public void sync(final FlowController controller, final DataFlow proposedFlow, final PropertyEncryptor encryptor, final FlowService flowService, final BundleUpdateStrategy bundleUpdateStrategy)
             throws FlowSerializationException, UninheritableFlowException, FlowSynchronizationException {
 
         final FlowManager flowManager = controller.getFlowManager();
@@ -217,7 +211,7 @@ public class StandardFlowSynchronizer implements FlowSynchronizer {
                 logger.debug("Proposed flow is not directly inheritable. However, the Controller has not been synchronized yet, " +
                     "so will check if the existing flow can be backed up and replaced by the proposed flow.");
 
-                final FlowInheritabilityCheck connectionMissingCheck = new ConnectionMissingCheck();
+                final FlowInheritabilityCheck connectionMissingCheck = new ConnectionMissingCheck(null);
                 final FlowInheritability connectionMissingInheritability = connectionMissingCheck.checkInheritability(existingDataFlow, proposedFlow, controller);
                 if (connectionMissingInheritability.isInheritable()) {
                     backupAndPurge = true;
@@ -530,7 +524,11 @@ public class StandardFlowSynchronizer implements FlowSynchronizer {
             .map(this::createParameter)
             .collect(Collectors.toMap(param -> param.getDescriptor().getName(), Function.identity()));
 
-        final ParameterContext context = flowManager.createParameterContext(dto.getId(), dto.getName(), parameters, dto.getInheritedParameterContexts());
+        final List<String> referencedIds = dto.getInheritedParameterContexts().stream()
+            .map(ParameterContextReferenceEntity::getId)
+            .collect(Collectors.toList());
+
+        final ParameterContext context = flowManager.createParameterContext(dto.getId(), dto.getName(), parameters, referencedIds);
         context.setDescription(dto.getDescription());
         return context;
     }
@@ -613,8 +611,7 @@ public class StandardFlowSynchronizer implements FlowSynchronizer {
         }
     }
 
-
-    private static boolean isFlowEmpty(final Document flowDocument) {
+    public static boolean isFlowEmpty(final Document flowDocument) {
         if (flowDocument == null) {
             return true;
         }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/inheritance/BundleCompatibilityCheck.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/inheritance/BundleCompatibilityCheck.java
index 504f582..e361636 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/inheritance/BundleCompatibilityCheck.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/inheritance/BundleCompatibilityCheck.java
@@ -16,24 +16,122 @@
  */
 package org.apache.nifi.controller.inheritance;
 
+import org.apache.nifi.bundle.BundleCoordinate;
 import org.apache.nifi.cluster.protocol.DataFlow;
 import org.apache.nifi.controller.FlowController;
+import org.apache.nifi.controller.flow.VersionedDataflow;
 import org.apache.nifi.controller.serialization.FlowFromDOMFactory;
+import org.apache.nifi.flow.Bundle;
+import org.apache.nifi.flow.VersionedControllerService;
+import org.apache.nifi.flow.VersionedProcessGroup;
+import org.apache.nifi.flow.VersionedProcessor;
+import org.apache.nifi.flow.VersionedReportingTask;
 import org.apache.nifi.nar.ExtensionManager;
 import org.apache.nifi.util.BundleUtils;
 import org.apache.nifi.util.DomUtils;
 import org.apache.nifi.web.api.dto.BundleDTO;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import org.w3c.dom.Document;
 import org.w3c.dom.Element;
 import org.w3c.dom.Node;
 import org.w3c.dom.NodeList;
 
+import java.util.Set;
+
 public class BundleCompatibilityCheck implements FlowInheritabilityCheck {
+    private static final Logger logger = LoggerFactory.getLogger(BundleCompatibilityCheck.class);
 
     @Override
     public FlowInheritability checkInheritability(final DataFlow existingFlow, final DataFlow proposedFlow, final FlowController flowController) {
-        final Document configuration = proposedFlow.getFlowDocument();
+        if (proposedFlow.isXml()) {
+            return checkInheritability(proposedFlow.getFlowDocument(), flowController);
+        } else {
+            return checkVersionedFlowInheritability(proposedFlow, flowController);
+        }
+    }
+
+    private FlowInheritability checkVersionedFlowInheritability(final DataFlow proposedFlow, final FlowController flowController) {
+        return checkBundles(proposedFlow, flowController.getExtensionManager());
+    }
+
+    private FlowInheritability checkBundles(final DataFlow proposedFlow, ExtensionManager extensionManager) {
+        final VersionedDataflow dataflow = proposedFlow.getVersionedDataflow();
+        if (dataflow == null) {
+            return FlowInheritability.inheritable();
+        }
+
+        final Set<String> missingComponents = proposedFlow.getMissingComponents();
+
+        if (dataflow.getControllerServices() != null) {
+            for (final VersionedControllerService service : dataflow.getControllerServices()) {
+                if (missingComponents.contains(service.getInstanceIdentifier())) {
+                    continue;
+                }
+
+                if (isMissing(service.getBundle(), extensionManager)) {
+                    return FlowInheritability.notInheritable(String.format("Controller Service with ID %s and type %s requires bundle %s, but that bundle cannot be found in this NiFi instance",
+                        service.getInstanceIdentifier(), service.getType(), service.getBundle()));
+                }
+            }
+        }
+
+        if (dataflow.getReportingTasks() != null) {
+            for (final VersionedReportingTask task : dataflow.getReportingTasks()) {
+                if (missingComponents.contains(task.getInstanceIdentifier())) {
+                    continue;
+                }
+
+                if (isMissing(task.getBundle(), extensionManager)) {
+                    return FlowInheritability.notInheritable(String.format("Reporting Task with ID %s and type %s requires bundle %s, but that bundle cannot be found in this NiFi instance",
+                        task.getInstanceIdentifier(), task.getType(), task.getBundle()));
+                }
+            }
+        }
+
+        return checkBundles(dataflow.getRootGroup(), extensionManager, missingComponents);
+    }
+
+    private FlowInheritability checkBundles(final VersionedProcessGroup group, final ExtensionManager extensionManager, final Set<String> missingComponents) {
+        for (final VersionedProcessor processor : group.getProcessors()) {
+            if (missingComponents.contains(processor.getInstanceIdentifier())) {
+                continue;
+            }
+
+            if (isMissing(processor.getBundle(), extensionManager)) {
+                return FlowInheritability.notInheritable(String.format("Processor with ID %s and type %s requires bundle %s, but that bundle cannot be found in this NiFi instance",
+                    processor.getInstanceIdentifier(), processor.getType(), processor.getBundle()));
+            }
+        }
+
+        for (final VersionedControllerService service : group.getControllerServices()) {
+            if (missingComponents.contains(service.getInstanceIdentifier())) {
+                continue;
+            }
+
+            if (isMissing(service.getBundle(), extensionManager)) {
+                return FlowInheritability.notInheritable(String.format("Controller Service with ID %s and type %s requires bundle %s, but that bundle cannot be found in this NiFi instance",
+                    service.getInstanceIdentifier(), service.getType(), service.getBundle()));
+            }
+        }
+
+        for (final VersionedProcessGroup childGroup : group.getProcessGroups()) {
+            final FlowInheritability childInheritability = checkBundles(childGroup, extensionManager, missingComponents);
+            if (!childInheritability.isInheritable()) {
+                return childInheritability;
+            }
+        }
+
+        return FlowInheritability.inheritable();
+    }
+
+    private boolean isMissing(final Bundle bundle, final ExtensionManager extensionManager) {
+        final BundleCoordinate coordinate = new BundleCoordinate(bundle.getGroup(), bundle.getArtifact(), bundle.getVersion());
+        final org.apache.nifi.bundle.Bundle existingBundle = extensionManager.getBundle(coordinate);
+        return existingBundle == null;
+    }
 
+    private FlowInheritability checkInheritability(final Document configuration, final FlowController flowController) {
         if (configuration == null) {
             return FlowInheritability.inheritable();
         }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/inheritance/ConnectionMissingCheck.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/inheritance/ConnectionMissingCheck.java
index 34fa87a..6832a03 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/inheritance/ConnectionMissingCheck.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/inheritance/ConnectionMissingCheck.java
@@ -17,11 +17,19 @@
 package org.apache.nifi.controller.inheritance;
 
 import org.apache.nifi.cluster.protocol.DataFlow;
+import org.apache.nifi.connectable.Connection;
 import org.apache.nifi.controller.FlowController;
+import org.apache.nifi.controller.flow.FlowManager;
+import org.apache.nifi.controller.flow.VersionedDataflow;
 import org.apache.nifi.controller.repository.FlowFileRepository;
 import org.apache.nifi.controller.serialization.FlowEncodingVersion;
 import org.apache.nifi.controller.serialization.FlowFromDOMFactory;
 import org.apache.nifi.controller.serialization.FlowSynchronizationException;
+import org.apache.nifi.flow.ComponentType;
+import org.apache.nifi.flow.VersionedComponent;
+import org.apache.nifi.registry.flow.diff.DifferenceType;
+import org.apache.nifi.registry.flow.diff.FlowComparison;
+import org.apache.nifi.registry.flow.diff.FlowDifference;
 import org.apache.nifi.web.api.dto.ConnectionDTO;
 import org.apache.nifi.web.api.dto.ProcessGroupDTO;
 import org.slf4j.Logger;
@@ -30,6 +38,7 @@ import org.w3c.dom.Document;
 import org.w3c.dom.Element;
 
 import java.io.IOException;
+import java.util.Collections;
 import java.util.HashSet;
 import java.util.Set;
 
@@ -40,9 +49,70 @@ import java.util.Set;
 public class ConnectionMissingCheck implements FlowInheritabilityCheck {
     private static final Logger logger = LoggerFactory.getLogger(ConnectionMissingCheck.class);
 
+    private final FlowComparison flowComparison;
+
+    public ConnectionMissingCheck(final FlowComparison flowComparison) {
+        this.flowComparison = flowComparison;
+    }
+
     @Override
     public FlowInheritability checkInheritability(final DataFlow existingFlow, final DataFlow proposedFlow, final FlowController flowController) {
-        final Document flowDocument = proposedFlow.getFlowDocument();
+        if (proposedFlow.isXml()) {
+            return checkInheritability(proposedFlow.getFlowDocument(), flowController);
+        } else {
+            return checkInheritability(existingFlow.getVersionedDataflow(), proposedFlow.getVersionedDataflow(), flowController);
+        }
+    }
+
+    private FlowInheritability checkInheritability(final VersionedDataflow existingFlow, final VersionedDataflow proposedFlow, final FlowController flowController) {
+        // Check for any connections that have been removed. The flow is not inheritable if any connection containing data is removed.
+        final FlowManager flowManager = flowController.getFlowManager();
+
+        // If FlowController has not yet been initialized, we need to determine which FlowFiles have queues by checking the FlowFile Repository.
+        // If the FlowController has already been initialized, we can just determine which connections have data by checking the connection itself.
+        final Set<String> queuesWithFlowFiles;
+        if (flowController.isInitialized()) {
+            queuesWithFlowFiles = Collections.emptySet();
+        } else {
+            final FlowFileRepository flowFileRepository = flowController.getRepositoryContextFactory().getFlowFileRepository();
+
+            try {
+                queuesWithFlowFiles = flowFileRepository.findQueuesWithFlowFiles(flowController.createSwapManager());
+            } catch (final IOException ioe) {
+                throw new FlowSynchronizationException("Failed to determine which connections have FlowFiles queued", ioe);
+            }
+
+            logger.debug("The following {} Connections/Queues have data queued up currently: {}", queuesWithFlowFiles.size(), queuesWithFlowFiles);
+        }
+
+        for (final FlowDifference difference : flowComparison.getDifferences()) {
+            final VersionedComponent component = difference.getComponentA();
+            if (DifferenceType.COMPONENT_REMOVED == difference.getDifferenceType() && component.getComponentType() == ComponentType.CONNECTION) {
+                // Flow Difference indicates that a connection was removed. Need to check if the Connection has data. Depending on whether or not the FlowController has been
+                // initialized, the method for doing this is either to check our Set Connection ID's that indicate that a Connection has data, or to just get the Connection itself
+                // and check if it has data.
+                if (queuesWithFlowFiles.contains(component.getInstanceIdentifier())) {
+                    return FlowInheritability.notInheritable("Inheriting cluster's flow would mean removing Connection with ID " + component.getInstanceIdentifier()
+                        + ", and the connection has data queued");
+                }
+
+                final Connection connection = flowManager.getConnection(component.getInstanceIdentifier());
+                if (connection == null) {
+                    continue;
+                }
+
+                final boolean queueEmpty = connection.getFlowFileQueue().isEmpty();
+                if (!queueEmpty) {
+                    return FlowInheritability.notInheritable("Inheriting cluster's flow would mean removing Connection with ID " + component.getInstanceIdentifier()
+                        + ", and the connection has data queued");
+                }
+            }
+        }
+
+        return FlowInheritability.inheritable();
+    }
+
+    private FlowInheritability checkInheritability(final Document flowDocument, final FlowController flowController) {
         final Element rootGroupElement = (Element) flowDocument.getDocumentElement().getElementsByTagName("rootGroup").item(0);
         final FlowEncodingVersion encodingVersion = FlowEncodingVersion.parse(flowDocument.getDocumentElement());
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/WriteAheadFlowFileRepository.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/WriteAheadFlowFileRepository.java
index 2af45c8..d8f07e0 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/WriteAheadFlowFileRepository.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/WriteAheadFlowFileRepository.java
@@ -830,7 +830,9 @@ public class WriteAheadFlowFileRepository implements FlowFileRepository, SyncLis
 
     @Override
     public Set<String> findQueuesWithFlowFiles(final FlowFileSwapManager swapManager) throws IOException {
-        recoveredRecords = wal.recoverRecords();
+        if (recoveredRecords == null) {
+            recoveredRecords = wal.recoverRecords();
+        }
 
         final Set<String> queueIds = new HashSet<>();
         for (final SerializedRepositoryRecord record : recoveredRecords) {
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/AffectedComponentSet.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/AffectedComponentSet.java
new file mode 100644
index 0000000..b62881e
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/AffectedComponentSet.java
@@ -0,0 +1,645 @@
+/*
+ * 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.serialization;
+
+import org.apache.nifi.connectable.Connectable;
+import org.apache.nifi.connectable.Connection;
+import org.apache.nifi.connectable.Port;
+import org.apache.nifi.controller.AbstractComponentNode;
+import org.apache.nifi.controller.ComponentNode;
+import org.apache.nifi.controller.FlowController;
+import org.apache.nifi.controller.ProcessorNode;
+import org.apache.nifi.controller.ReportingTaskNode;
+import org.apache.nifi.controller.ScheduledState;
+import org.apache.nifi.controller.UninheritableFlowException;
+import org.apache.nifi.controller.flow.FlowManager;
+import org.apache.nifi.controller.service.ControllerServiceNode;
+import org.apache.nifi.controller.service.ControllerServiceProvider;
+import org.apache.nifi.controller.service.ControllerServiceState;
+import org.apache.nifi.flow.ComponentType;
+import org.apache.nifi.flow.ConnectableComponent;
+import org.apache.nifi.flow.ConnectableComponentType;
+import org.apache.nifi.flow.VersionedComponent;
+import org.apache.nifi.flow.VersionedConnection;
+import org.apache.nifi.groups.ProcessGroup;
+import org.apache.nifi.groups.RemoteProcessGroup;
+import org.apache.nifi.parameter.ParameterContext;
+import org.apache.nifi.registry.flow.diff.DifferenceType;
+import org.apache.nifi.registry.flow.diff.FlowDifference;
+import org.apache.nifi.remote.RemoteGroupPort;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.stream.Collectors;
+
+/**
+ * <p>
+ *     An AffectedComponentSet is a collection of components that will need to be enabled/disabled/started/stopped in order to facilitate
+ *     some set of changes to a dataflow.
+ * </p>
+ */
+public class AffectedComponentSet {
+    private static final Logger logger = LoggerFactory.getLogger(AffectedComponentSet.class);
+    private final FlowController flowController;
+    private final FlowManager flowManager;
+
+    private final Set<Port> inputPorts = new HashSet<>();
+    private final Set<Port> outputPorts = new HashSet<>();
+    private final Set<RemoteGroupPort> remoteInputPorts = new HashSet<>();
+    private final Set<RemoteGroupPort> remoteOutputPorts = new HashSet<>();
+    private final Set<ProcessorNode> processors = new HashSet<>();
+    private final Set<ControllerServiceNode> controllerServices = new HashSet<>();
+    private final Set<ReportingTaskNode> reportingTasks = new HashSet<>();
+
+    public AffectedComponentSet(final FlowController flowController) {
+        this.flowController = flowController;
+        this.flowManager = flowController.getFlowManager();
+    }
+
+    public void addInputPort(final Port port) {
+        if (port == null) {
+            return;
+        }
+
+        inputPorts.add(port);
+    }
+
+    public void addOutputPort(final Port port) {
+        if (port == null) {
+            return;
+        }
+
+        outputPorts.add(port);
+    }
+
+    public void addRemoteInputPort(final RemoteGroupPort port) {
+        if (port == null) {
+            return;
+        }
+
+        remoteInputPorts.add(port);
+    }
+
+    public void addRemoteOutputPort(final RemoteGroupPort port) {
+        if (port == null) {
+            return;
+        }
+
+        remoteOutputPorts.add(port);
+    }
+
+    public void addRemoteProcessGroup(final RemoteProcessGroup remoteProcessGroup) {
+        if (remoteProcessGroup == null) {
+            return;
+        }
+
+        remoteProcessGroup.getInputPorts().forEach(this::addRemoteInputPort);
+        remoteProcessGroup.getOutputPorts().forEach(this::addRemoteOutputPort);
+    }
+
+    public void addProcessor(final ProcessorNode processor) {
+        if (processor == null) {
+            return;
+        }
+
+        processors.add(processor);
+    }
+
+    public void addControllerService(final ControllerServiceNode controllerService) {
+        if (controllerService == null) {
+            return;
+        }
+
+        controllerServices.add(controllerService);
+
+        final List<ComponentNode> referencingComponents = controllerService.getReferences().findRecursiveReferences(ComponentNode.class);
+        for (final ComponentNode reference : referencingComponents) {
+            if (reference instanceof ControllerServiceNode) {
+                addControllerService((ControllerServiceNode) reference);
+            } else if (reference instanceof ProcessorNode) {
+                addProcessor((ProcessorNode) reference);
+            } else if (reference instanceof ReportingTaskNode) {
+                addReportingTask((ReportingTaskNode) reference);
+            }
+        }
+    }
+
+    public boolean isControllerServiceAffected(final String serviceId) {
+        for (final ControllerServiceNode serviceNode : controllerServices) {
+            if (serviceNode.getIdentifier().equals(serviceId)) {
+                return true;
+            }
+        }
+
+        return false;
+    }
+
+    private void addControllerServiceWithoutReferences(final ControllerServiceNode controllerService) {
+        if (controllerService == null) {
+            return;
+        }
+
+        controllerServices.add(controllerService);
+    }
+
+    public void addReportingTask(final ReportingTaskNode task) {
+        if (task == null) {
+            return;
+        }
+
+        reportingTasks.add(task);
+    }
+
+    public boolean isReportingTaskAffected(final String reportingTaskId) {
+        for (final ReportingTaskNode taskNode : reportingTasks) {
+            if (taskNode.getIdentifier().equals(reportingTaskId)) {
+                return true;
+            }
+        }
+
+        return false;
+    }
+
+    public void addConnection(final Connection connection) {
+        if (connection == null) {
+            return;
+        }
+
+        addConnectable(connection.getSource());
+        addConnectable(connection.getDestination());
+    }
+
+    public void addConnectable(final Connectable connectable) {
+        if (connectable == null) {
+            return;
+        }
+
+        switch (connectable.getConnectableType()) {
+            case INPUT_PORT:
+                addInputPort((Port) connectable);
+                break;
+            case OUTPUT_PORT:
+                addOutputPort((Port) connectable);
+                break;
+            case PROCESSOR:
+                addProcessor((ProcessorNode) connectable);
+                break;
+            case REMOTE_INPUT_PORT:
+                addRemoteInputPort((RemoteGroupPort) connectable);
+                break;
+            case REMOTE_OUTPUT_PORT:
+                addRemoteOutputPort((RemoteGroupPort) connectable);
+        }
+    }
+
+    /**
+     * Adds any component that is affected by the given Flow Difference
+     * @param difference the Flow Difference
+     */
+    public void addAffectedComponents(final FlowDifference difference) {
+        final DifferenceType differenceType = difference.getDifferenceType();
+
+        if (differenceType == DifferenceType.COMPONENT_ADDED) {
+            // The component doesn't exist. But if it's a connection, the source or the destination might. And those need to be accounted for.
+            if (difference.getComponentB().getComponentType() == ComponentType.CONNECTION) {
+                addComponentsForNewConnection((VersionedConnection) difference.getComponentB());
+            }
+
+            return;
+        }
+
+        if (differenceType == DifferenceType.PARAMETER_VALUE_CHANGED || differenceType == DifferenceType.PARAMETER_DESCRIPTION_CHANGED || differenceType == DifferenceType.PARAMETER_REMOVED) {
+            addComponentsForParameterUpdate(difference);
+            return;
+        }
+
+        if (differenceType == DifferenceType.PARAMETER_CONTEXT_CHANGED) {
+            addComponentsForParameterContextChange(difference);
+            return;
+        }
+
+        if (differenceType == DifferenceType.INHERITED_CONTEXTS_CHANGED) {
+            addComponentsForInheritedParameterContextChange(difference);
+        }
+
+        if (differenceType == DifferenceType.VARIABLE_CHANGED || differenceType == DifferenceType.VARIABLE_ADDED || differenceType == DifferenceType.VARIABLE_REMOVED) {
+            addComponentsForVariableChange(difference.getComponentA().getInstanceIdentifier(), difference.getFieldName().orElse(null));
+            return;
+        }
+
+        if (differenceType == DifferenceType.RPG_URL_CHANGED) {
+            final String instanceId = difference.getComponentA().getInstanceIdentifier();
+            final RemoteProcessGroup rpg = flowManager.getRootGroup().findRemoteProcessGroup(instanceId);
+            if (rpg != null) {
+                addRemoteProcessGroup(rpg);
+            }
+        }
+
+        if (differenceType == DifferenceType.COMPONENT_REMOVED && difference.getComponentA().getComponentType() == ComponentType.PROCESS_GROUP) {
+            // If a Process Group is removed, we need to consider any component within the Process Group as affected also
+            addAllComponentsWithinGroup(difference.getComponentA().getInstanceIdentifier());
+        }
+
+        addAffectedComponents(difference.getComponentA());
+    }
+
+    private void addAllComponentsWithinGroup(final String groupId) {
+        final ProcessGroup processGroup = flowManager.getGroup(groupId);
+        if (processGroup == null) {
+            return;
+        }
+
+        processGroup.getProcessors().forEach(this::addProcessor);
+        processGroup.getControllerServices(false).forEach(this::addControllerServiceWithoutReferences);
+        processGroup.getInputPorts().forEach(this::addInputPort);
+        processGroup.getOutputPorts().forEach(this::addOutputPort);
+        processGroup.getRemoteProcessGroups().forEach(this::addRemoteProcessGroup);
+        processGroup.getProcessGroups().forEach(child -> addAllComponentsWithinGroup(child.getIdentifier()));
+    }
+
+    private void addComponentsForVariableChange(final String groupId, final String variableName) {
+        if (groupId == null || variableName == null) {
+            return;
+        }
+
+        final ProcessGroup group = flowManager.getGroup(groupId);
+        if (group == null) {
+            return;
+        }
+
+        final Set<ComponentNode> affectedComponents = group.getComponentsAffectedByVariable(variableName);
+        for (final ComponentNode component : affectedComponents) {
+            if (component instanceof ProcessorNode) {
+                addProcessor((ProcessorNode) component);
+            } else if (component instanceof ControllerServiceNode) {
+                addControllerService((ControllerServiceNode) component);
+            }
+        }
+    }
+
+    private void addComponentsForInheritedParameterContextChange(final FlowDifference difference) {
+        // If the inherited parameter contexts have changed, any component referencing a parameter in that context is affected.
+        final String parameterContextId = difference.getComponentA().getInstanceIdentifier();
+        final ParameterContext context = flowManager.getParameterContextManager().getParameterContext(parameterContextId);
+        if (context == null) {
+            return;
+        }
+
+        final Set<ProcessGroup> boundGroups = context.getParameterReferenceManager().getProcessGroupsBound(context);
+        for (final ProcessGroup group : boundGroups) {
+            group.getProcessors().stream()
+                .filter(AbstractComponentNode::isReferencingParameter)
+                .forEach(this::addProcessor);
+
+            group.getControllerServices(false).stream()
+                .filter(ComponentNode::isReferencingParameter)
+                .forEach(this::addControllerService);
+        }
+    }
+
+    private void addComponentsForParameterContextChange(final FlowDifference difference) {
+        // When the parameter context that a PG is bound to is updated, any component referencing a parameter is affected.
+        final String groupId = difference.getComponentA().getInstanceIdentifier();
+        final ProcessGroup group = flowManager.getGroup(groupId);
+        if (group == null) {
+            return;
+        }
+
+        group.getProcessors().stream()
+            .filter(AbstractComponentNode::isReferencingParameter)
+            .forEach(this::addProcessor);
+
+        group.getControllerServices(false).stream()
+            .filter(ComponentNode::isReferencingParameter)
+            .forEach(this::addControllerService);
+    }
+
+    private void addComponentsForParameterUpdate(final FlowDifference difference) {
+        final DifferenceType differenceType = difference.getDifferenceType();
+
+        final Optional<String> optionalParameterName = difference.getFieldName();
+        if (!optionalParameterName.isPresent()) {
+            logger.warn("Encountered a Flow Difference {} with Difference Type of {} but no indication as to which parameter was updated.", difference, differenceType);
+            return;
+        }
+
+        final String parameterName = optionalParameterName.get();
+        final String contextId = difference.getComponentA().getInstanceIdentifier();
+        final ParameterContext parameterContext = flowManager.getParameterContextManager().getParameterContext(contextId);
+        if (parameterContext == null) {
+            logger.warn("Encountered a Flow Difference {} with a Difference Type of {} but found no Parameter Context with Instance ID {}", difference, differenceType, contextId);
+            return;
+        }
+
+        final Set<ControllerServiceNode> referencingServices = parameterContext.getParameterReferenceManager().getControllerServicesReferencing(parameterContext, parameterName);
+        final Set<ProcessorNode> referencingProcessors = parameterContext.getParameterReferenceManager().getProcessorsReferencing(parameterContext, parameterName);
+
+        referencingServices.forEach(this::addControllerService);
+        referencingProcessors.forEach(this::addProcessor);
+    }
+
+    private void addComponentsForNewConnection(final VersionedConnection connection) {
+        final ConnectableComponent sourceComponent = connection.getSource();
+        final Connectable sourceConnectable = getConnectable(sourceComponent.getType(), sourceComponent.getInstanceIdentifier());
+        if (sourceConnectable != null) {
+            addConnectable(sourceConnectable);
+        }
+
+        final ConnectableComponent destinationComponent = connection.getDestination();
+        final Connectable destinationConnectable = getConnectable(destinationComponent.getType(), destinationComponent.getInstanceIdentifier());
+        if (destinationConnectable != null) {
+            addConnectable(destinationConnectable);
+        }
+    }
+
+    private Connectable getConnectable(final ConnectableComponentType type, final String identifier) {
+        switch (type) {
+            case FUNNEL:
+                return flowManager.getFunnel(identifier);
+            case INPUT_PORT:
+                return flowManager.getInputPort(identifier);
+            case OUTPUT_PORT:
+                return flowManager.getOutputPort(identifier);
+            case PROCESSOR:
+                return flowManager.getProcessorNode(identifier);
+            case REMOTE_INPUT_PORT:
+            case REMOTE_OUTPUT_PORT:
+                return flowManager.getRootGroup().findRemoteGroupPort(identifier);
+            default:
+                return null;
+        }
+    }
+
+    private void addAffectedComponents(final VersionedComponent versionedComponent) {
+        final String componentId = versionedComponent.getInstanceIdentifier();
+        switch (versionedComponent.getComponentType()) {
+            case CONNECTION:
+                addConnection(flowManager.getConnection(componentId));
+                break;
+            case CONTROLLER_SERVICE:
+                addControllerService(flowManager.getControllerServiceNode(componentId));
+                break;
+            case INPUT_PORT:
+                addInputPort(flowManager.getInputPort(componentId));
+                break;
+            case OUTPUT_PORT:
+                addOutputPort(flowManager.getOutputPort(componentId));
+                break;
+            case PROCESS_GROUP:
+                break;
+            case PROCESSOR:
+                addProcessor(flowManager.getProcessorNode(componentId));
+                break;
+            case REMOTE_INPUT_PORT:
+                final RemoteGroupPort remoteInputPort = flowManager.getRootGroup().findRemoteGroupPort(componentId);
+                if (remoteInputPort != null) {
+                    addRemoteInputPort(remoteInputPort);
+                }
+                break;
+            case REMOTE_OUTPUT_PORT:
+                final RemoteGroupPort remoteOutputPort = flowManager.getRootGroup().findRemoteGroupPort(componentId);
+                if (remoteOutputPort != null) {
+                    addRemoteOutputPort(remoteOutputPort);
+                }
+                break;
+            case REMOTE_PROCESS_GROUP:
+                addRemoteProcessGroup(flowManager.getRootGroup().findRemoteProcessGroup(componentId));
+                break;
+            case REPORTING_TASK:
+                addReportingTask(flowManager.getReportingTaskNode(componentId));
+                break;
+        }
+    }
+
+    /**
+     * Returns a new AffectedComponentSet that represents only those components that are currently active. A component is considered active if it is an Input/Output Port
+     * and is running, is a Processor or reporting task that has at least one active thread or a scheduled state of RUNNING or STARTING, or is a Controller Service that is
+     * ENABLED or ENABLING.
+     *
+     * @return an AffectedComponentSet that represents all components within this AffectedComponentSet that are currently active. The components contained by the returned AffectedComponentSet
+     * will always be a subset or equal to the set of components contained by this.
+     */
+    public AffectedComponentSet toActiveSet() {
+        final AffectedComponentSet active = new AffectedComponentSet(flowController);
+        inputPorts.stream().filter(port -> port.getScheduledState() == ScheduledState.RUNNING).forEach(active::addInputPort);
+        outputPorts.stream().filter(port -> port.getScheduledState() == ScheduledState.RUNNING).forEach(active::addOutputPort);
+        remoteInputPorts.stream().filter(port -> port.getScheduledState() == ScheduledState.RUNNING).forEach(active::addRemoteInputPort);
+        remoteOutputPorts.stream().filter(port -> port.getScheduledState() == ScheduledState.RUNNING).forEach(active::addRemoteOutputPort);
+
+        processors.stream().filter(this::isActive).forEach(active::addProcessor);
+        reportingTasks.stream().filter(task -> task.getScheduledState() == ScheduledState.STARTING || task.getScheduledState() == ScheduledState.RUNNING || task.isRunning())
+            .forEach(active::addReportingTask);
+        controllerServices.stream().filter(service -> service.getState() == ControllerServiceState.ENABLING || service.getState() == ControllerServiceState.ENABLED)
+            .forEach(active::addControllerServiceWithoutReferences);
+
+        return active;
+    }
+
+    private boolean isActive(final ProcessorNode processor) {
+        // We consider component active if it's starting, running, or has active threads. The call to ProcessorNode.isRunning() will only return true if it has active threads or a scheduled
+        // state of RUNNING but not if it has a scheduled state of STARTING.
+        final ScheduledState scheduledState = processor.getPhysicalScheduledState();
+        return scheduledState == ScheduledState.STARTING || scheduledState == ScheduledState.RUNNING || processor.isRunning();
+    }
+
+    private boolean isStopped(final ProcessorNode processor) {
+        final ScheduledState state = processor.getPhysicalScheduledState();
+        final boolean stateCorrect = state == ScheduledState.STOPPED || state == ScheduledState.DISABLED;
+        return stateCorrect && !processor.isRunning();
+    }
+
+    public void start() {
+        logger.info("Starting the following components: {}", this);
+        flowController.getControllerServiceProvider().enableControllerServices(controllerServices);
+
+        inputPorts.forEach(port -> port.getProcessGroup().startInputPort(port));
+        outputPorts.forEach(port -> port.getProcessGroup().startOutputPort(port));
+        remoteInputPorts.forEach(port -> port.getRemoteProcessGroup().startTransmitting(port));
+        remoteOutputPorts.forEach(port -> port.getRemoteProcessGroup().startTransmitting(port));
+        processors.forEach(processor -> processor.getProcessGroup().startProcessor(processor, false));
+        reportingTasks.forEach(flowController::startReportingTask);
+    }
+
+    /**
+     * Returns a new AffectedComponentSet that represents only those components that currently exist within the NiFi instance. When a set of dataflow updates have occurred, it is very possible
+     * that one or more components referred to by the AffectedComponentSet no longer exist (for example, there was a dataflow update that removed a Processor, so that Processor no longer exists).
+     *
+     * @return an AffectedComponentSet that represents all components within this AffectedComponentSet that currently exist within the NiFi instance. The components contained by the returned
+     * AffectedComponentSetwill always be a subset or equal to the set of components contained by this.
+     */
+    public AffectedComponentSet toExistingSet() {
+        final ControllerServiceProvider serviceProvider = flowController.getControllerServiceProvider();
+
+        final AffectedComponentSet existing = new AffectedComponentSet(flowController);
+        inputPorts.stream().filter(port -> port.getProcessGroup().getInputPort(port.getIdentifier()) != null).forEach(existing::addInputPort);
+        outputPorts.stream().filter(port -> port.getProcessGroup().getOutputPort(port.getIdentifier()) != null).forEach(existing::addOutputPort);
+        remoteInputPorts.stream().filter(port -> port.getProcessGroup().findRemoteGroupPort(port.getIdentifier()) != null).forEach(existing::addRemoteInputPort);
+        remoteOutputPorts.stream().filter(port -> port.getProcessGroup().findRemoteGroupPort(port.getIdentifier()) != null).forEach(existing::addRemoteOutputPort);
+        processors.stream().filter(processor -> processor.getProcessGroup().getProcessor(processor.getIdentifier()) != null).forEach(existing::addProcessor);
+        reportingTasks.stream().filter(task -> flowController.getReportingTaskNode(task.getIdentifier()) != null).forEach(existing::addReportingTask);
+        controllerServices.stream().filter(service -> serviceProvider.getControllerServiceNode(service.getIdentifier()) != null).forEach(existing::addControllerServiceWithoutReferences);
+
+        return existing;
+    }
+
+
+    /**
+     * Returns a new AffectedComponentSet that represents only those components that currently can be started. When a set of dataflow updates have occurred, it is very possible
+     * that one or more components referred to by the AffectedComponentSet can no longer be started (for example, there was a dataflow update that disabled a Processor that previously was running).
+     *
+     * @return an AffectedComponentSet that represents all components within this AffectedComponentSet that currently exist within the NiFi instance. The components contained by the returned
+     * AffectedComponentSet will always be a subset or equal to the set of components contained by this.
+     */
+    public AffectedComponentSet toStartableSet() {
+        final AffectedComponentSet startable = new AffectedComponentSet(flowController);
+        inputPorts.stream().filter(this::isStartable).forEach(startable::addInputPort);
+        outputPorts.stream().filter(this::isStartable).forEach(startable::addOutputPort);
+        remoteInputPorts.stream().filter(this::isStartable).forEach(startable::addRemoteInputPort);
+        remoteOutputPorts.stream().filter(this::isStartable).forEach(startable::addRemoteOutputPort);
+        processors.stream().filter(this::isStartable).forEach(startable::addProcessor);
+        reportingTasks.stream().filter(this::isStartable).forEach(startable::addReportingTask);
+        controllerServices.stream().filter(this::isStartable).forEach(startable::addControllerServiceWithoutReferences);
+
+        return startable;
+    }
+
+    private boolean isStartable(final ComponentNode componentNode) {
+        if (componentNode == null) {
+            return false;
+        }
+
+        if (componentNode instanceof ProcessorNode) {
+            return ((ProcessorNode) componentNode).getScheduledState() != ScheduledState.DISABLED;
+        }
+        if (componentNode instanceof ReportingTaskNode) {
+            return ((ReportingTaskNode) componentNode).getScheduledState() != ScheduledState.DISABLED;
+        }
+
+        return true;
+    }
+
+    private boolean isStartable(final Port port) {
+        if (port == null) {
+            return false;
+        }
+
+        return port.getScheduledState() != ScheduledState.DISABLED;
+    }
+
+    public void stop() {
+        logger.info("Stopping the following components: {}", this);
+        final long start = System.currentTimeMillis();
+
+        inputPorts.forEach(port -> port.getProcessGroup().stopInputPort(port));
+        outputPorts.forEach(port -> port.getProcessGroup().stopOutputPort(port));
+        remoteInputPorts.forEach(port -> port.getRemoteProcessGroup().stopTransmitting(port));
+        remoteOutputPorts.forEach(port -> port.getRemoteProcessGroup().stopTransmitting(port));
+        processors.forEach(processor -> processor.getProcessGroup().stopProcessor(processor));
+        reportingTasks.forEach(flowController::stopReportingTask);
+
+        waitForConnectablesStopped();
+
+        if (!controllerServices.isEmpty()) {
+            final Future<Void> disableFuture = flowController.getControllerServiceProvider().disableControllerServicesAsync(controllerServices);
+            waitForControllerServicesStopped(disableFuture);
+        }
+
+        final long millis = System.currentTimeMillis() - start;
+        logger.info("Successfully stopped all components in {} milliseconds", millis);
+    }
+
+    private void waitForControllerServicesStopped(final Future<Void> future) {
+        try {
+            while (true) {
+                logger.info("Waiting for all Controller Services to become disabled...");
+                if (logger.isDebugEnabled()) {
+                    final Set<ControllerServiceNode> activeServices = controllerServices.stream().filter(ControllerServiceNode::isActive).collect(Collectors.toSet());
+                    logger.debug("There are currently {} active Controller Services: {}", activeServices.size(), activeServices);
+                }
+
+                try {
+                    future.get(10, TimeUnit.SECONDS);
+                    return;
+                } catch (final TimeoutException ignored) {
+                }
+            }
+        } catch (final Exception e) {
+            throw new UninheritableFlowException("Could not disable all affected Controller Services", e);
+        }
+    }
+
+    private void waitForConnectablesStopped() {
+        long count = 0L;
+        try {
+            while (!componentsStopped()) {
+                if (count++ % 1000 == 0) {
+                    // The 0th time and every 1000th time (10 seconds), log an update
+                    logger.info("Waiting for all required Processors and Reporting Tasks to stop...");
+                    if (reportingTasks.isEmpty() && processors.isEmpty()) {
+                        return;
+                    }
+
+                    if (logger.isDebugEnabled()) {
+                        final Set<ReportingTaskNode> activeReportingTasks = reportingTasks.stream().filter(ReportingTaskNode::isRunning).collect(Collectors.toSet());
+                        logger.debug("There are currently {} active Reporting Tasks: {}", activeReportingTasks.size(), activeReportingTasks);
+
+                        final Set<ProcessorNode> activeProcessors = processors.stream()
+                            .filter(processor -> !isStopped(processor))
+                            .collect(Collectors.toSet());
+                        logger.debug("There are currently {} active Processors: {}", activeProcessors.size(), activeProcessors);
+                    }
+                }
+
+                Thread.sleep(10L);
+            }
+        } catch (final Exception e) {
+            throw new UninheritableFlowException("Could not stop all affected components", e);
+        }
+    }
+
+    private boolean componentsStopped() {
+        if (processors.stream().anyMatch(processor -> !isStopped(processor))) {
+            return false;
+        }
+        if (reportingTasks.stream().anyMatch(ReportingTaskNode::isRunning)) {
+            return false;
+        }
+
+        return true;
+    }
+
+    @Override
+    public String toString() {
+        return "AffectedComponentSet[" +
+            "inputPorts=" + inputPorts +
+            ", outputPorts=" + outputPorts +
+            ", remoteInputPorts=" + remoteInputPorts +
+            ", remoteOutputPorts=" + remoteOutputPorts +
+            ", processors=" + processors +
+            ", controllerServices=" + controllerServices +
+            ", reportingTasks=" + reportingTasks +
+            "]";
+    }
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/FlowSynchronizer.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/FlowSynchronizer.java
index 80a1cb5..7746e22 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/FlowSynchronizer.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/FlowSynchronizer.java
@@ -21,6 +21,7 @@ import org.apache.nifi.controller.FlowController;
 import org.apache.nifi.controller.MissingBundleException;
 import org.apache.nifi.controller.UninheritableFlowException;
 import org.apache.nifi.encrypt.PropertyEncryptor;
+import org.apache.nifi.groups.BundleUpdateStrategy;
 import org.apache.nifi.services.FlowService;
 
 /**
@@ -43,7 +44,7 @@ public interface FlowSynchronizer {
      * @throws FlowSynchronizationException if updates to the controller failed. If this exception is thrown, then the controller should be considered unsafe to be used
      * @throws MissingBundleException if the proposed flow cannot be loaded by the controller because it contains a bundle that is not available to the controller
      */
-    void sync(FlowController controller, DataFlow dataFlow, PropertyEncryptor encryptor, FlowService flowService)
+    void sync(FlowController controller, DataFlow dataFlow, PropertyEncryptor encryptor, FlowService flowService, BundleUpdateStrategy bundleUpdateStrategy)
             throws FlowSerializationException, UninheritableFlowException, FlowSynchronizationException, MissingBundleException;
 
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/ScheduledStateLookup.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/ScheduledStateLookup.java
index 4a6b420..d3f03b5 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/ScheduledStateLookup.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/ScheduledStateLookup.java
@@ -27,7 +27,8 @@ public interface ScheduledStateLookup {
 
     ScheduledState getScheduledState(Port port);
 
-    public static final ScheduledStateLookup IDENTITY_LOOKUP = new ScheduledStateLookup() {
+
+    ScheduledStateLookup IDENTITY_LOOKUP = new ScheduledStateLookup() {
         @Override
         public ScheduledState getScheduledState(final ProcessorNode procNode) {
             return procNode.getDesiredState();
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/StandardFlowSynchronizer.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/StandardFlowSynchronizer.java
new file mode 100644
index 0000000..1ef9831
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/StandardFlowSynchronizer.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.controller.serialization;
+
+import org.apache.nifi.cluster.protocol.DataFlow;
+import org.apache.nifi.controller.FlowController;
+import org.apache.nifi.controller.MissingBundleException;
+import org.apache.nifi.controller.UninheritableFlowException;
+import org.apache.nifi.controller.XmlFlowSynchronizer;
+import org.apache.nifi.encrypt.PropertyEncryptor;
+import org.apache.nifi.groups.BundleUpdateStrategy;
+import org.apache.nifi.services.FlowService;
+
+public class StandardFlowSynchronizer implements FlowSynchronizer {
+    private final XmlFlowSynchronizer xmlFlowSynchronizer;
+    private final VersionedFlowSynchronizer versionedFlowSynchronizer;
+
+    public StandardFlowSynchronizer(final XmlFlowSynchronizer xmlFlowSynchronizer, final VersionedFlowSynchronizer versionedFlowSynchronizer) {
+        this.xmlFlowSynchronizer = xmlFlowSynchronizer;
+        this.versionedFlowSynchronizer = versionedFlowSynchronizer;
+    }
+
+    @Override
+    public void sync(final FlowController controller, final DataFlow dataFlow, final PropertyEncryptor encryptor, final FlowService flowService, final BundleUpdateStrategy bundleUpdateStrategy)
+        throws FlowSerializationException, UninheritableFlowException, FlowSynchronizationException, MissingBundleException {
+
+        final FlowSynchronizer synchronizer = isXml(dataFlow) ? xmlFlowSynchronizer : versionedFlowSynchronizer;
+        synchronizer.sync(controller, dataFlow, encryptor, flowService, bundleUpdateStrategy);
+    }
+
+    public static boolean isFlowEmpty(final DataFlow dataFlow) {
+        if (dataFlow == null || dataFlow.getFlow() == null || dataFlow.getFlow().length == 0) {
+            return true;
+        }
+
+        if (isXml(dataFlow)) {
+            return XmlFlowSynchronizer.isFlowEmpty(dataFlow.getFlowDocument());
+        } else {
+            return VersionedFlowSynchronizer.isFlowEmpty(dataFlow);
+        }
+    }
+
+    private static boolean isXml(final DataFlow dataFlow) {
+        if (dataFlow == null || dataFlow.getFlow() == null || dataFlow.getFlow().length == 0) {
+            return true;
+        }
+
+        return dataFlow.isXml();
+    }
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/VersionedDataflowMapper.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/VersionedDataflowMapper.java
new file mode 100644
index 0000000..2c7e784
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/VersionedDataflowMapper.java
@@ -0,0 +1,221 @@
+/*
+ * 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.serialization;
+
+import org.apache.nifi.connectable.Port;
+import org.apache.nifi.controller.FlowController;
+import org.apache.nifi.controller.ProcessorNode;
+import org.apache.nifi.controller.ReportingTaskNode;
+import org.apache.nifi.controller.Template;
+import org.apache.nifi.controller.flow.VersionedDataflow;
+import org.apache.nifi.controller.flow.VersionedFlowEncodingVersion;
+import org.apache.nifi.controller.flow.VersionedRegistry;
+import org.apache.nifi.controller.flow.VersionedTemplate;
+import org.apache.nifi.controller.service.ControllerServiceNode;
+import org.apache.nifi.flow.ScheduledState;
+import org.apache.nifi.flow.VersionedControllerService;
+import org.apache.nifi.flow.VersionedProcessGroup;
+import org.apache.nifi.flow.VersionedReportingTask;
+import org.apache.nifi.groups.ProcessGroup;
+import org.apache.nifi.nar.ExtensionManager;
+import org.apache.nifi.parameter.ParameterContext;
+import org.apache.nifi.registry.flow.FlowRegistry;
+import org.apache.nifi.registry.flow.FlowRegistryClient;
+import org.apache.nifi.registry.flow.VersionedParameterContext;
+import org.apache.nifi.registry.flow.mapping.ComponentIdLookup;
+import org.apache.nifi.registry.flow.mapping.FlowMappingOptions;
+import org.apache.nifi.registry.flow.mapping.NiFiRegistryFlowMapper;
+import org.apache.nifi.registry.flow.mapping.SensitiveValueEncryptor;
+import org.apache.nifi.registry.flow.mapping.VersionedComponentStateLookup;
+import org.apache.nifi.web.api.dto.TemplateDTO;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+public class VersionedDataflowMapper {
+    private static final VersionedFlowEncodingVersion ENCODING_VERSION = new VersionedFlowEncodingVersion(2, 0);
+
+    private final FlowController flowController;
+    private final NiFiRegistryFlowMapper flowMapper;
+    private final ScheduledStateLookup stateLookup;
+
+    public VersionedDataflowMapper(final FlowController flowController, final ExtensionManager extensionManager, final SensitiveValueEncryptor encryptor, final ScheduledStateLookup stateLookup) {
+        this.flowController = flowController;
+        this.stateLookup = stateLookup;
+
+        final VersionedComponentStateLookup versionedComponentStateLookup = createStateLookup();
+
+        final FlowMappingOptions mappingOptions = new FlowMappingOptions.Builder()
+            .mapSensitiveConfiguration(true)
+            .mapPropertyDescriptors(false)
+            .stateLookup(versionedComponentStateLookup)
+            .sensitiveValueEncryptor(encryptor)
+            .componentIdLookup(ComponentIdLookup.VERSIONED_OR_GENERATE)
+            .mapInstanceIdentifiers(true)
+            .mapControllerServiceReferencesToVersionedId(false)
+            .build();
+
+        flowMapper = new NiFiRegistryFlowMapper(extensionManager, mappingOptions);
+    }
+
+    public VersionedDataflow createMapping() {
+        final VersionedDataflow dataflow = new VersionedDataflow();
+        dataflow.setEncodingVersion(ENCODING_VERSION);
+        dataflow.setMaxTimerDrivenThreadCount(flowController.getMaxTimerDrivenThreadCount());
+        dataflow.setControllerServices(mapControllerServices());
+        dataflow.setParameterContexts(mapParameterContexts());
+        dataflow.setRegistries(mapRegistries());
+        dataflow.setReportingTasks(mapReportingTasks());
+        dataflow.setRootGroup(mapRootGroup());
+        dataflow.setTemplates(mapTemplates());
+
+        return dataflow;
+    }
+
+    private List<VersionedControllerService> mapControllerServices() {
+        final List<VersionedControllerService> controllerServices = new ArrayList<>();
+
+        for (final ControllerServiceNode serviceNode : flowController.getFlowManager().getRootControllerServices()) {
+            final VersionedControllerService versionedControllerService = flowMapper.mapControllerService(
+                serviceNode, flowController.getControllerServiceProvider(), Collections.emptySet(), Collections.emptyMap());
+            controllerServices.add(versionedControllerService);
+        }
+
+        return controllerServices;
+    }
+
+    private List<VersionedParameterContext> mapParameterContexts() {
+        final List<VersionedParameterContext> parameterContexts = new ArrayList<>();
+
+        for (final ParameterContext parameterContext : flowController.getFlowManager().getParameterContextManager().getParameterContexts()) {
+            final VersionedParameterContext versionedParameterContext = flowMapper.mapParameterContext(parameterContext);
+            parameterContexts.add(versionedParameterContext);
+        }
+
+        return parameterContexts;
+    }
... 6246 lines suppressed ...