You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by mc...@apache.org on 2020/04/09 15:17:44 UTC

[nifi] branch master updated: NIFI-6849: Reworked how nodes inherit cluster information when joining a cluster. Now, if there are conflicts, a local copy is made of the flow/authorizations/etc. and the cluster's flow is inherited. - Refactored Flow Synchronization to make code cleaner - Updated Authorizers to forcibly inherit Users, Groups, and Access Policies if the local flow is empty. - Updated FlowFileRepositories to use SerializedRepositoryRecord instead of RepositoryRecord, so that we have the ability to read [...]

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 683b1d9  NIFI-6849: Reworked how nodes inherit cluster information when joining a cluster. Now, if there are conflicts, a local copy is made of the flow/authorizations/etc. and the cluster's flow is inherited.  - Refactored Flow Synchronization to make code cleaner  - Updated Authorizers to forcibly inherit Users, Groups, and Access Policies if the local flow is empty.  - Updated FlowFileRepositories to use SerializedRepositoryRecord instead of RepositoryRecord, so that we have t [...]
683b1d9 is described below

commit 683b1d995200384dcde8ec269163bc57f25b23ce
Author: Mark Payne <ma...@hotmail.com>
AuthorDate: Mon Nov 4 15:21:50 2019 -0500

    NIFI-6849: Reworked how nodes inherit cluster information when joining a cluster. Now, if there are conflicts, a local copy is made of the flow/authorizations/etc. and the cluster's flow is inherited.
     - Refactored Flow Synchronization to make code cleaner
     - Updated Authorizers to forcibly inherit Users, Groups, and Access Policies if the local flow is empty.
     - Updated FlowFileRepositories to use SerializedRepositoryRecord instead of RepositoryRecord, so that we have the ability to read records without already knowing the Queue objects. Updated StandardFlowSynchronizer so that if the flow is not inheritable but the controller has not yet been initialized, the flow is backed up and replaced instead of NiFi failing to start
    - Added system tests. Updated FlowController so that if it fails to inherit flow due to flow uninheritability that it notifies the cluster of this instead of remaining in the 'CONNECTING' state.
    - Added additional log statements to aid in debugging
    
    NIFI-6849: Rebased against master. Updated Admin Guide to describe new cluster flow inheritance behavior
    
    NIFI-6849: Addressed review feedback
    
    NIFI-6849: Addressed review feedback: Relocated logic for bundle compatibility into the BundleCompatibilityCheck class. Fixed logic that prevented users/groups/policies from being forcibly inherited during startup
    
    This closes #3891
---
 .../src/main/asciidoc/administration-guide.adoc    |   27 +-
 .../AbstractPolicyBasedAuthorizer.java             |   76 +-
 .../ConfigurableAccessPolicyProvider.java          |    9 +
 .../ConfigurableUserGroupProvider.java             |    9 +
 .../nifi/authorization/ManagedAuthorizer.java      |    9 +
 .../controller/repository/FlowFileRepository.java  |   10 +-
 .../authorization/MockPolicyBasedAuthorizer.java   |   10 +
 .../nifi/authorization/AuthorizerFactory.java      |   15 +
 .../authorization/FileAccessPolicyProvider.java    |   92 +-
 .../apache/nifi/authorization/FileAuthorizer.java  |   18 +-
 .../nifi/authorization/FileUserGroupProvider.java  |  140 ++-
 .../EncryptedRepositoryRecordSerdeFactory.java     |    4 +-
 .../EncryptedSchemaRepositoryRecordSerde.java      |   26 +-
 .../ReconstitutedSerializedRepositoryRecord.java   |  111 +++
 .../repository/RepositoryRecordSerde.java          |   26 +-
 .../repository/RepositoryRecordSerdeFactory.java   |    9 +-
 .../repository/SchemaRepositoryRecordSerde.java    |   78 +-
 .../repository/SerializedRepositoryRecord.java     |   62 ++
 .../StandardRepositoryRecordSerdeFactory.java      |   30 +-
 .../WriteAheadRepositoryRecordSerde.java           |  137 +--
 .../schema/RepositoryRecordFieldMap.java           |   18 +-
 .../SchemaRepositoryRecordSerdeTest.java           |  292 ------
 .../CompositeConfigurableUserGroupProvider.java    |    5 +
 .../authorization/StandardManagedAuthorizer.java   |   13 +
 .../SimpleConfigurableUserGroupProvider.java       |    5 +
 .../nifi/cluster/protocol/StandardDataFlow.java    |   63 +-
 .../org/apache/nifi/cluster/protocol/DataFlow.java |   12 +-
 .../apache/nifi/controller/flow/FlowManager.java   |   14 +
 .../serialization/FlowSerializationException.java  |    0
 .../java/org/apache/nifi/groups/ProcessGroup.java  |   14 +
 .../java/org/apache/nifi/services/FlowService.java |   32 +-
 .../org/apache/nifi/controller/FlowController.java |   84 +-
 .../nifi/controller/StandardFlowService.java       |   51 +-
 .../nifi/controller/StandardFlowSynchronizer.java  | 1037 +++++++++-----------
 .../nifi/controller/flow/StandardFlowManager.java  |   52 +
 .../controller/inheritance/AuthorizerCheck.java    |   75 ++
 .../inheritance/BundleCompatibilityCheck.java      |   81 ++
 .../inheritance/ConnectionMissingCheck.java        |   87 ++
 .../inheritance/FlowFingerprintCheck.java          |   97 ++
 .../controller/inheritance/FlowInheritability.java |   68 ++
 .../inheritance/FlowInheritabilityCheck.java}      |   35 +-
 .../inheritance/MissingComponentsCheck.java        |   51 +
 .../repository/LiveSerializedRepositoryRecord.java |   79 ++
 .../repository/RocksDBFlowFileRepository.java      |   69 +-
 .../repository/VolatileFlowFileRepository.java     |    7 +
 .../repository/WriteAheadFlowFileRepository.java   |  133 ++-
 .../controller/serialization/FlowSynchronizer.java |    4 +-
 .../nifi/fingerprint/FingerprintFactory.java       |   44 +-
 .../apache/nifi/groups/StandardProcessGroup.java   |   11 +-
 .../logging/repository/StandardLogRepository.java  |    2 +-
 .../nifi/persistence/FlowConfigurationDAO.java     |   12 +-
 .../StandardXMLFlowConfigurationDAO.java           |   25 +-
 .../serialization/FlowFromDOMFactoryTest.groovy    |   81 --
 ...cryptedSequentialAccessWriteAheadLogTest.groovy |   63 +-
 .../apache/nifi/controller/TestFlowController.java |  153 ++-
 .../SchemaRepositoryRecordSerdeTest.java           |   67 +-
 .../repository/TestStandardProcessSession.java     |    5 +
 .../TestWriteAheadFlowFileRepository.java          |    8 +-
 .../controller/service/mock/MockProcessGroup.java  |    5 +
 .../nifi/integration/FrameworkIntegrationTest.java |    3 +-
 .../OOMEWriteAheadFlowFileRepository.java          |   38 +-
 .../authorization/MockPolicyBasedAuthorizer.java   |   12 +-
 .../nifi/web/revision/NaiveRevisionManager.java    |   15 +-
 .../authorization/ManagedRangerAuthorizer.java     |   11 +-
 nifi-system-tests/nifi-system-test-suite/pom.xml   |   14 +
 .../nifi/tests/system/AggregateNiFiInstance.java   |   11 +-
 .../apache/nifi/tests/system/NiFiClientUtil.java   |   23 +-
 .../org/apache/nifi/tests/system/NiFiInstance.java |   12 +-
 .../org/apache/nifi/tests/system/NiFiSystemIT.java |    6 +-
 .../SpawnedStandaloneNiFiInstanceFactory.java      |   20 +-
 .../JoinCluslterWithMissingConnectionWithData.java |  141 +++
 .../clustering/JoinClusterWithDifferentFlow.java   |  253 +++++
 .../JoinClusterWithMissingConnectionNoData.java    |   48 +
 .../resources/conf/clustered/node2/bootstrap.conf  |    2 +-
 .../resources/conf/clustered/node2/logback.xml     |    2 +
 .../src/test/resources/conf/default/logback.xml    |    1 -
 .../resources/flows/mismatched-flows/flow1.xml.gz  |  Bin 0 -> 3553 bytes
 .../resources/flows/mismatched-flows/flow2.xml.gz  |  Bin 0 -> 3530 bytes
 .../missing-connection/with-connection.xml.gz      |  Bin 0 -> 1031 bytes
 .../missing-connection/without-connection.xml.gz   |  Bin 0 -> 770 bytes
 80 files changed, 2869 insertions(+), 1595 deletions(-)

diff --git a/nifi-docs/src/main/asciidoc/administration-guide.adoc b/nifi-docs/src/main/asciidoc/administration-guide.adoc
index b992cfd..874b85f 100644
--- a/nifi-docs/src/main/asciidoc/administration-guide.adoc
+++ b/nifi-docs/src/main/asciidoc/administration-guide.adoc
@@ -1704,17 +1704,34 @@ For more information, see the <<toolkit-guide.adoc#nifi_CLI,NiFi CLI>> section i
 === Flow Election
 When a cluster first starts up, NiFi must determine which of the nodes have the
 "correct" version of the flow. This is done by voting on the flows that each of the nodes has. When a node
-attempts to connect to a cluster, it provides a copy of its local flow to the Cluster Coordinator. If no flow
+attempts to connect to a cluster, it provides a copy of its local flow and (if the policy provider allows for configuration via NiFi)
+its users, groups, and policies, to the Cluster Coordinator. If no flow
 has yet been elected the "correct" flow, the node's flow is compared to each of the other Nodes' flows. If another
 Node's flow matches this one, a vote is cast for this flow. If no other Node has reported the same flow yet, this
 flow will be added to the pool of possibly elected flows with one vote. After
 some amount of time has elapsed (configured by setting the `nifi.cluster.flow.election.max.wait.time` property) or
 some number of Nodes have cast votes (configured by setting the `nifi.cluster.flow.election.max.candidates` property),
-a flow is elected to be the "correct" copy of the flow. All nodes that have incompatible flows are then disconnected
-from the cluster while those with compatible flows inherit the cluster's flow. 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
+a flow is elected to be the "correct" copy of the flow.
+
+Any node whose dataflow, users, groups, and policies conflict the with those elected will backup any conflicting resources and replace the local
+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`.
+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.
+
+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
+dataflow. If that queue does not exist in the elected dataflow, the node will not inherit the dataflow, users, groups, and policies. Instead, NiFi will
+log errors to that effect and will fail to startup. This ensures that even if the node has data stored in a connection, and the cluster's dataflow is different,
+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
-not be voted to be the "correct" flow unless no other flow is found.
+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.
+
 
 === Basic Cluster Setup
 
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 929e2ad..7500717 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,6 +20,8 @@ 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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import org.w3c.dom.Document;
 import org.w3c.dom.Element;
 import org.w3c.dom.Node;
@@ -46,6 +48,7 @@ import java.util.Set;
  * An Authorizer that provides management of users, groups, and policies.
  */
 public abstract class AbstractPolicyBasedAuthorizer implements ManagedAuthorizer {
+    private static final Logger logger = LoggerFactory.getLogger(AbstractPolicyBasedAuthorizer.class);
 
     static final DocumentBuilderFactory DOCUMENT_BUILDER_FACTORY = DocumentBuilderFactory.newInstance();
     static final XMLOutputFactory XML_OUTPUT_FACTORY = XMLOutputFactory.newInstance();
@@ -149,6 +152,10 @@ public abstract class AbstractPolicyBasedAuthorizer implements ManagedAuthorizer
      */
     public abstract Group getGroup(String identifier) throws AuthorizationAccessException;
 
+    protected abstract void purgePoliciesUsersAndGroups();
+
+    protected abstract void backupPoliciesUsersAndGroups();
+
     /**
      * The group represented by the provided instance will be updated based on the provided instance.
      *
@@ -337,23 +344,23 @@ public abstract class AbstractPolicyBasedAuthorizer implements ManagedAuthorizer
      */
     @Override
     public final void checkInheritability(String proposedFingerprint) throws AuthorizationAccessException, UninheritableAuthorizationsException {
+        final PoliciesUsersAndGroups policiesUsersAndGroups;
         try {
             // ensure we understand the proposed fingerprint
-            parsePoliciesUsersAndGroups(proposedFingerprint);
+            policiesUsersAndGroups = parsePoliciesUsersAndGroups(proposedFingerprint);
         } catch (final AuthorizationAccessException e) {
             throw new UninheritableAuthorizationsException("Unable to parse proposed fingerprint: " + e);
         }
 
-        final List<User> users = getSortedUsers();
-        final List<Group> groups = getSortedGroups();
-        final List<AccessPolicy> accessPolicies = getSortedAccessPolicies();
-
-        // ensure we're in a state to inherit
-        if (!users.isEmpty() || !groups.isEmpty() || !accessPolicies.isEmpty()) {
-            throw new UninheritableAuthorizationsException("Proposed fingerprint is not inheritable because the current Authorizations is not empty..");
+        if (!isInheritable(policiesUsersAndGroups)) {
+            throw new UninheritableAuthorizationsException("Proposed fingerprint is not inheritable because the current Authorizations is not empty.");
         }
     }
 
+    private boolean isInheritable(final PoliciesUsersAndGroups policiesUsersAndGroups) {
+        return getUsers().isEmpty() && getGroups().isEmpty() && getAccessPolicies().isEmpty();
+    }
+
     /**
      * Parses the fingerprint and adds any users, groups, and policies to the current Authorizer.
      *
@@ -366,9 +373,39 @@ public abstract class AbstractPolicyBasedAuthorizer implements ManagedAuthorizer
         }
 
         final PoliciesUsersAndGroups policiesUsersAndGroups = parsePoliciesUsersAndGroups(fingerprint);
-        policiesUsersAndGroups.getUsers().forEach(user -> addUser(user));
-        policiesUsersAndGroups.getGroups().forEach(group -> addGroup(group));
-        policiesUsersAndGroups.getAccessPolicies().forEach(policy -> addAccessPolicy(policy));
+        inheritPoliciesUsersAndGroups(policiesUsersAndGroups);
+    }
+
+    private void inheritPoliciesUsersAndGroups(final PoliciesUsersAndGroups policiesUsersAndGroups) {
+        addPoliciesUsersAndGroups(policiesUsersAndGroups);
+    }
+
+    private void addPoliciesUsersAndGroups(final PoliciesUsersAndGroups policiesUsersAndGroups) {
+        policiesUsersAndGroups.getUsers().forEach(this::addUser);
+        policiesUsersAndGroups.getGroups().forEach(this::addGroup);
+        policiesUsersAndGroups.getAccessPolicies().forEach(this::addAccessPolicy);
+    }
+
+    @Override
+    public void forciblyInheritFingerprint(final String fingerprint) throws AuthorizationAccessException {
+        if (fingerprint == null || fingerprint.trim().isEmpty()) {
+            logger.info("Inheriting Empty Policies, Users & Groups. Will backup existing Policies, Users & Groups first.");
+            backupPoliciesUsersAndGroups();
+            purgePoliciesUsersAndGroups();
+
+            return;
+        }
+
+        final PoliciesUsersAndGroups policiesUsersAndGroups = parsePoliciesUsersAndGroups(fingerprint);
+        if (isInheritable(policiesUsersAndGroups)) {
+            logger.debug("Inheriting Polciies, Users & Groups");
+            inheritPoliciesUsersAndGroups(policiesUsersAndGroups);
+        } else {
+            logger.info("Cannot directly inherit Policies, Users & Groups. Will backup existing Policies, Users & Groups, and then replace with proposed configuration");
+            backupPoliciesUsersAndGroups();
+            purgePoliciesUsersAndGroups();
+            addPoliciesUsersAndGroups(policiesUsersAndGroups);
+        }
     }
 
     private PoliciesUsersAndGroups parsePoliciesUsersAndGroups(final String fingerprint) {
@@ -507,6 +544,11 @@ public abstract class AbstractPolicyBasedAuthorizer implements ManagedAuthorizer
             }
 
             @Override
+            public void forciblyInheritFingerprint(final String fingerprint) throws AuthorizationAccessException {
+                throw new UnsupportedOperationException();
+            }
+
+            @Override
             public void checkInheritability(String proposedFingerprint) throws AuthorizationAccessException, UninheritableAuthorizationsException {
                 // fingerprint is managed by the encapsulating class
                 throw new UnsupportedOperationException();
@@ -602,6 +644,12 @@ public abstract class AbstractPolicyBasedAuthorizer implements ManagedAuthorizer
                     }
 
                     @Override
+                    public void forciblyInheritFingerprint(final String fingerprint) throws AuthorizationAccessException {
+                        // fingerprint is managed by the encapsulating class
+                        throw new UnsupportedOperationException();
+                    }
+
+                    @Override
                     public void checkInheritability(String proposedFingerprint) throws AuthorizationAccessException, UninheritableAuthorizationsException {
                         // fingerprint is managed by the encapsulating class
                         throw new UnsupportedOperationException();
@@ -738,19 +786,19 @@ public abstract class AbstractPolicyBasedAuthorizer implements ManagedAuthorizer
 
     private List<AccessPolicy> getSortedAccessPolicies() {
         final List<AccessPolicy> policies = new ArrayList<>(getAccessPolicies());
-        Collections.sort(policies, Comparator.comparing(AccessPolicy::getIdentifier));
+        policies.sort(Comparator.comparing(AccessPolicy::getIdentifier));
         return policies;
     }
 
     private List<Group> getSortedGroups() {
         final List<Group> groups = new ArrayList<>(getGroups());
-        Collections.sort(groups, Comparator.comparing(Group::getIdentifier));
+        groups.sort(Comparator.comparing(Group::getIdentifier));
         return groups;
     }
 
     private List<User> getSortedUsers() {
         final List<User> users = new ArrayList<>(getUsers());
-        Collections.sort(users, Comparator.comparing(User::getIdentifier));
+        users.sort(Comparator.comparing(User::getIdentifier));
         return users;
     }
 
diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/authorization/ConfigurableAccessPolicyProvider.java b/nifi-framework-api/src/main/java/org/apache/nifi/authorization/ConfigurableAccessPolicyProvider.java
index 1ec4dde..d6dc0e7 100644
--- a/nifi-framework-api/src/main/java/org/apache/nifi/authorization/ConfigurableAccessPolicyProvider.java
+++ b/nifi-framework-api/src/main/java/org/apache/nifi/authorization/ConfigurableAccessPolicyProvider.java
@@ -47,6 +47,15 @@ public interface ConfigurableAccessPolicyProvider extends AccessPolicyProvider {
     void inheritFingerprint(final String fingerprint) throws AuthorizationAccessException;
 
     /**
+     * Parses the fingerprint and determines whether or not the fingerprint can be inherited in the same manner as {@link #inheritFingerprint(String)}. If so, will inherit as such.
+     * Otherwise, a backup of the existing policy provider will be made, if possible, and the policies will be replaced with those in the given fingerprint.
+     *
+     * @param fingerprint the fingerprint to replace the existing policies with
+     * @throws AuthorizationAccessException if unable to perform the operation
+     */
+    void forciblyInheritFingerprint(final String fingerprint) throws AuthorizationAccessException;
+
+    /**
      * When the fingerprints are not equal, this method will check if the proposed fingerprint is inheritable.
      * If the fingerprint is an exact match, this method will not be invoked as there is nothing to inherit.
      *
diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/authorization/ConfigurableUserGroupProvider.java b/nifi-framework-api/src/main/java/org/apache/nifi/authorization/ConfigurableUserGroupProvider.java
index ad592aa..0d2d733 100644
--- a/nifi-framework-api/src/main/java/org/apache/nifi/authorization/ConfigurableUserGroupProvider.java
+++ b/nifi-framework-api/src/main/java/org/apache/nifi/authorization/ConfigurableUserGroupProvider.java
@@ -47,6 +47,15 @@ public interface ConfigurableUserGroupProvider extends UserGroupProvider {
     void inheritFingerprint(final String fingerprint) throws AuthorizationAccessException;
 
     /**
+     * Parses the fingerprint and determines whether or not the fingerprint can be inherited in the same manner as {@link #inheritFingerprint(String)}. If so, will inherit as such.
+     * Otherwise, a backup of the existing user group provider will be made, if possible, and the policies will be replaced with those in the given fingerprint.
+     *
+     * @param fingerprint the fingerprint to replace the existing policies with
+     * @throws AuthorizationAccessException if unable to perform the operation
+     */
+    void forciblyInheritFingerprint(final String fingerprint) throws AuthorizationAccessException;
+
+    /**
      * When the fingerprints are not equal, this method will check if the proposed fingerprint is inheritable.
      * If the fingerprint is an exact match, this method will not be invoked as there is nothing to inherit.
      *
diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/authorization/ManagedAuthorizer.java b/nifi-framework-api/src/main/java/org/apache/nifi/authorization/ManagedAuthorizer.java
index d70ee55..53f5e5f 100644
--- a/nifi-framework-api/src/main/java/org/apache/nifi/authorization/ManagedAuthorizer.java
+++ b/nifi-framework-api/src/main/java/org/apache/nifi/authorization/ManagedAuthorizer.java
@@ -40,6 +40,15 @@ public interface ManagedAuthorizer extends Authorizer {
     void inheritFingerprint(final String fingerprint) throws AuthorizationAccessException;
 
     /**
+     * Parses the fingerprint and determines whether or not the fingerprint can be inherited in the same manner as {@link #inheritFingerprint(String)}. If so, will inherit as such.
+     * Otherwise, a backup of the existing policy provider will be made, if possible, and the policies will be replaced with those in the given fingerprint.
+     *
+     * @param fingerprint the fingerprint to replace the existing policies with
+     * @throws AuthorizationAccessException if unable to perform the operation
+     */
+    void forciblyInheritFingerprint(final String fingerprint) throws AuthorizationAccessException;
+
+    /**
      * When the fingerprints are not equal, this method will check if the proposed fingerprint is inheritable.
      * If the fingerprint is an exact match, this method will not be invoked as there is nothing to inherit.
      *
diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/controller/repository/FlowFileRepository.java b/nifi-framework-api/src/main/java/org/apache/nifi/controller/repository/FlowFileRepository.java
index 1da1107..0a7bad7 100644
--- a/nifi-framework-api/src/main/java/org/apache/nifi/controller/repository/FlowFileRepository.java
+++ b/nifi-framework-api/src/main/java/org/apache/nifi/controller/repository/FlowFileRepository.java
@@ -88,6 +88,13 @@ public interface FlowFileRepository extends Closeable {
     long loadFlowFiles(QueueProvider queueProvider) throws IOException;
 
     /**
+     * Searches through the repository to find the ID's of all FlowFile Queues that currently have data queued
+     * @return the set of all FlowFileQueue identifiers for which a FlowFile is queued
+     * @throws IOException if unable to read from the FlowFile Repository
+     */
+    Set<String> findQueuesWithFlowFiles(FlowFileSwapManager flowFileSwapManager) throws IOException;
+
+    /**
      * @return <code>true</code> if the Repository is volatile (i.e., its data
      * is lost upon application restart), <code>false</code> otherwise
      */
@@ -167,7 +174,8 @@ public interface FlowFileRepository extends Closeable {
      * @return a Mapping of Resource Claim to a representation of the FlowFiles/Swap Files that reference those Resource Claims
      * @throws IOException if an IO failure occurs when attempting to find references
      */
-    default Map<ResourceClaim, Set<ResourceClaimReference>> findResourceClaimReferences(Set<ResourceClaim> resourceClaims, FlowFileSwapManager swapManager) throws IOException {
+    default Map<ResourceClaim, Set<ResourceClaimReference>> findResourceClaimReferences(Set<ResourceClaim> resourceClaims, FlowFileSwapManager swapManager)
+        throws IOException {
         return null;
     }
 }
diff --git a/nifi-framework-api/src/test/java/org/apache/nifi/authorization/MockPolicyBasedAuthorizer.java b/nifi-framework-api/src/test/java/org/apache/nifi/authorization/MockPolicyBasedAuthorizer.java
index 9b50b50..5d80f12 100644
--- a/nifi-framework-api/src/test/java/org/apache/nifi/authorization/MockPolicyBasedAuthorizer.java
+++ b/nifi-framework-api/src/test/java/org/apache/nifi/authorization/MockPolicyBasedAuthorizer.java
@@ -61,6 +61,16 @@ public class MockPolicyBasedAuthorizer extends AbstractPolicyBasedAuthorizer {
     }
 
     @Override
+    protected void purgePoliciesUsersAndGroups() {
+
+    }
+
+    @Override
+    protected void backupPoliciesUsersAndGroups() {
+
+    }
+
+    @Override
     public Group doUpdateGroup(Group group) throws AuthorizationAccessException {
         deleteGroup(group);
         return addGroup(group);
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-authorizer/src/main/java/org/apache/nifi/authorization/AuthorizerFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-authorizer/src/main/java/org/apache/nifi/authorization/AuthorizerFactory.java
index b826cf1..933b235 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-authorizer/src/main/java/org/apache/nifi/authorization/AuthorizerFactory.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-authorizer/src/main/java/org/apache/nifi/authorization/AuthorizerFactory.java
@@ -129,6 +129,11 @@ public final class AuthorizerFactory {
                 }
 
                 @Override
+                public void forciblyInheritFingerprint(final String fingerprint) throws AuthorizationAccessException {
+                    baseManagedAuthorizer.forciblyInheritFingerprint(fingerprint);
+                }
+
+                @Override
                 public void checkInheritability(String proposedFingerprint) throws AuthorizationAccessException, UninheritableAuthorizationsException {
                     baseManagedAuthorizer.checkInheritability(proposedFingerprint);
                 }
@@ -150,6 +155,11 @@ public final class AuthorizerFactory {
                             }
 
                             @Override
+                            public void forciblyInheritFingerprint(final String fingerprint) throws AuthorizationAccessException {
+                                baseConfigurableAccessPolicyProvider.forciblyInheritFingerprint(fingerprint);
+                            }
+
+                            @Override
                             public void checkInheritability(String proposedFingerprint) throws AuthorizationAccessException, UninheritableAuthorizationsException {
                                 baseConfigurableAccessPolicyProvider.checkInheritability(proposedFingerprint);
                             }
@@ -215,6 +225,11 @@ public final class AuthorizerFactory {
                                         }
 
                                         @Override
+                                        public void forciblyInheritFingerprint(final String fingerprint) throws AuthorizationAccessException {
+                                            baseConfigurableAccessPolicyProvider.forciblyInheritFingerprint(fingerprint);
+                                        }
+
+                                        @Override
                                         public void checkInheritability(String proposedFingerprint) throws AuthorizationAccessException, UninheritableAuthorizationsException {
                                             baseConfigurableUserGroupProvider.checkInheritability(proposedFingerprint);
                                         }
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 d29a669..8064d94 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
@@ -64,6 +64,8 @@ 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;
@@ -277,21 +279,52 @@ public class FileAccessPolicyProvider implements ConfigurableAccessPolicyProvide
 
     @Override
     public synchronized AccessPolicy addAccessPolicy(AccessPolicy accessPolicy) throws AuthorizationAccessException {
-        if (accessPolicy == null) {
-            throw new IllegalArgumentException("AccessPolicy cannot be null");
-        }
+        addAccessPolicies(Collections.singletonList(accessPolicy));
+        return authorizationsHolder.get().getPoliciesById().get(accessPolicy.getIdentifier());
+    }
 
-        // create the new JAXB Policy
-        final Policy policy = createJAXBPolicy(accessPolicy);
+    private synchronized void addAccessPolicies(final List<AccessPolicy> accessPolicies) throws AuthorizationAccessException {
+        if (accessPolicies == null) {
+            throw new IllegalArgumentException("AccessPolicies cannot be null");
+        }
 
-        // add the new Policy to the top-level list of policies
         final AuthorizationsHolder holder = authorizationsHolder.get();
         final Authorizations authorizations = holder.getAuthorizations();
-        authorizations.getPolicies().getPolicy().add(policy);
+        final List<Policy> policyList = authorizations.getPolicies().getPolicy();
+
+        for (final AccessPolicy accessPolicy : accessPolicies) {
+            // create the new JAXB Policy
+            final Policy policy = createJAXBPolicy(accessPolicy);
+
+            // add the new Policy to the top-level list of policies
+            policyList.add(policy);
+        }
 
         saveAndRefreshHolder(authorizations);
+    }
 
-        return authorizationsHolder.get().getPoliciesById().get(accessPolicy.getIdentifier());
+    public synchronized void purgePolicies(final boolean save) {
+        final AuthorizationsHolder holder = authorizationsHolder.get();
+        final Authorizations authorizations = holder.getAuthorizations();
+        final List<Policy> policyList = authorizations.getPolicies().getPolicy();
+
+        policyList.clear();
+
+        if (save) {
+            saveAndRefreshHolder(authorizations);
+        }
+    }
+
+    public void backupPolicies() throws JAXBException {
+        final AuthorizationsHolder holder = authorizationsHolder.get();
+        final Authorizations authorizations = holder.getAuthorizations();
+
+        final DateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd-HH-mm-ss");
+        final String timestamp = dateFormat.format(new Date());
+
+        final File backupFile = new File(authorizationsFile.getParentFile(), authorizationsFile.getName() + "." + timestamp);
+        logger.info("Writing backup of Policies to {}", backupFile.getAbsolutePath());
+        saveAuthorizations(authorizations, backupFile);
     }
 
     @Override
@@ -380,24 +413,55 @@ public class FileAccessPolicyProvider implements ConfigurableAccessPolicyProvide
 
     @Override
     public synchronized void inheritFingerprint(String fingerprint) throws AuthorizationAccessException {
-        parsePolicies(fingerprint).forEach(policy -> addAccessPolicy(policy));
+        final List<AccessPolicy> accessPolicies = parsePolicies(fingerprint);
+        inheritAccessPolicies(accessPolicies);
+    }
+
+    private synchronized void inheritAccessPolicies(final List<AccessPolicy> accessPolicies) {
+        addAccessPolicies(accessPolicies);
+    }
+
+    @Override
+    public synchronized void forciblyInheritFingerprint(final String fingerprint) throws AuthorizationAccessException {
+        final List<AccessPolicy> accessPolicies = parsePolicies(fingerprint);
+
+        if (isInheritable(accessPolicies)) {
+            logger.debug("Inheriting cluster's Access Policies");
+            inheritAccessPolicies(accessPolicies);
+        } else {
+            logger.info("Cannot directly inherit cluster's Access Policies. Will create backup of existing policies and replace with proposed policies");
+
+            try {
+                backupPolicies();
+            } catch (final JAXBException jaxb) {
+                throw new AuthorizationAccessException("Failed to backup existing policies so will not inherit any policies", jaxb);
+            }
+
+            purgePolicies(false);
+            addAccessPolicies(accessPolicies);
+        }
     }
 
     @Override
     public void checkInheritability(String proposedFingerprint) throws AuthorizationAccessException, UninheritableAuthorizationsException {
+        final List<AccessPolicy> accessPolicies;
         try {
             // ensure we can understand the proposed fingerprint
-            parsePolicies(proposedFingerprint);
+            accessPolicies = parsePolicies(proposedFingerprint);
         } catch (final AuthorizationAccessException e) {
             throw new UninheritableAuthorizationsException("Unable to parse the proposed fingerprint: " + e);
         }
 
         // ensure we are in a proper state to inherit the fingerprint
-        if (!getAccessPolicies().isEmpty()) {
+        if (!isInheritable(accessPolicies)) {
             throw new UninheritableAuthorizationsException("Proposed fingerprint is not inheritable because the current access policies is not empty.");
         }
     }
 
+    private boolean isInheritable(final List<AccessPolicy> accessPolicies) {
+        return getAccessPolicies().isEmpty();
+    }
+
     @Override
     public String getFingerprint() throws AuthorizationAccessException {
         final List<AccessPolicy> policies = new ArrayList<>(getAccessPolicies());
@@ -555,10 +619,14 @@ public class FileAccessPolicyProvider implements ConfigurableAccessPolicyProvide
     }
 
     private void saveAuthorizations(final Authorizations authorizations) throws JAXBException {
+        saveAuthorizations(authorizations, authorizationsFile);
+    }
+
+    private void saveAuthorizations(final Authorizations authorizations, final File destinationFile) throws JAXBException {
         final Marshaller marshaller = JAXB_AUTHORIZATIONS_CONTEXT.createMarshaller();
         marshaller.setSchema(authorizationsSchema);
         marshaller.setProperty(Marshaller.JAXB_FORMATTED_OUTPUT, true);
-        marshaller.marshal(authorizations, authorizationsFile);
+        marshaller.marshal(authorizations, destinationFile);
     }
 
     private Authorizations unmarshallAuthorizations() throws JAXBException {
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/main/java/org/apache/nifi/authorization/FileAuthorizer.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/main/java/org/apache/nifi/authorization/FileAuthorizer.java
index b64a36a..6ab1643 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/main/java/org/apache/nifi/authorization/FileAuthorizer.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/main/java/org/apache/nifi/authorization/FileAuthorizer.java
@@ -23,6 +23,7 @@ import org.apache.nifi.util.NiFiProperties;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import javax.xml.bind.JAXBException;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Set;
@@ -228,6 +229,22 @@ public class FileAuthorizer extends AbstractPolicyBasedAuthorizer {
         return accessPolicyProvider.getAccessPolicies();
     }
 
+    @Override
+    public void purgePoliciesUsersAndGroups() {
+        accessPolicyProvider.purgePolicies(true);
+        userGroupProvider.purgeUsersAndGroups();
+    }
+
+    @Override
+    public void backupPoliciesUsersAndGroups() {
+        try {
+            accessPolicyProvider.backupPolicies();
+            userGroupProvider.backupUsersAndGroups();
+        } catch (final JAXBException jaxb) {
+            throw new AuthorizationAccessException("Failed to backup policies", jaxb);
+        }
+    }
+
     @AuthorizerContext
     public void setNiFiProperties(NiFiProperties properties) {
         userGroupProvider.setNiFiProperties(properties);
@@ -256,5 +273,4 @@ public class FileAuthorizer extends AbstractPolicyBasedAuthorizer {
             }
         };
     }
-
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/main/java/org/apache/nifi/authorization/FileUserGroupProvider.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/main/java/org/apache/nifi/authorization/FileUserGroupProvider.java
index 5ff2d49..6492449 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/main/java/org/apache/nifi/authorization/FileUserGroupProvider.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/main/java/org/apache/nifi/authorization/FileUserGroupProvider.java
@@ -60,6 +60,8 @@ 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;
@@ -199,7 +201,7 @@ public class FileUserGroupProvider implements ConfigurableUserGroupProvider {
             }
 
             logger.info(String.format("Users/Groups file loaded at %s", new Date().toString()));
-        } catch (IOException | AuthorizerCreationException | JAXBException | IllegalStateException | SAXException e) {
+        } catch (IOException | AuthorizerCreationException | JAXBException | IllegalStateException e) {
             throw new AuthorizerCreationException(e);
         }
     }
@@ -215,13 +217,8 @@ public class FileUserGroupProvider implements ConfigurableUserGroupProvider {
             throw new IllegalArgumentException("User cannot be null");
         }
 
-        final org.apache.nifi.authorization.file.tenants.generated.User jaxbUser = createJAXBUser(user);
-
-        final UserGroupHolder holder = userGroupHolder.get();
-        final Tenants tenants = holder.getTenants();
-        tenants.getUsers().getUser().add(jaxbUser);
-
-        saveAndRefreshHolder(tenants);
+        final UsersAndGroups usersAndGroups = new UsersAndGroups(Collections.singletonList(user), Collections.emptyList());
+        addUsersAndGroups(usersAndGroups);
 
         return userGroupHolder.get().getUsersById().get(user.getIdentifier());
     }
@@ -325,30 +322,44 @@ public class FileUserGroupProvider implements ConfigurableUserGroupProvider {
         return userGroupHolder.get().getAllGroups();
     }
 
-    @Override
-    public synchronized Group addGroup(Group group) throws AuthorizationAccessException {
-        if (group == null) {
-            throw new IllegalArgumentException("Group cannot be null");
-        }
-
+    private synchronized void addUsersAndGroups(final UsersAndGroups usersAndGroups) {
         final UserGroupHolder holder = userGroupHolder.get();
         final Tenants tenants = holder.getTenants();
 
-        // create a new JAXB Group based on the incoming Group
-        final org.apache.nifi.authorization.file.tenants.generated.Group jaxbGroup = new org.apache.nifi.authorization.file.tenants.generated.Group();
-        jaxbGroup.setIdentifier(group.getIdentifier());
-        jaxbGroup.setName(group.getName());
+        final List<Group> groups = usersAndGroups.getGroups();
+        for (final Group group : groups) {
+            // create a new JAXB Group based on the incoming Group
+            final org.apache.nifi.authorization.file.tenants.generated.Group jaxbGroup = new org.apache.nifi.authorization.file.tenants.generated.Group();
+            jaxbGroup.setIdentifier(group.getIdentifier());
+            jaxbGroup.setName(group.getName());
+
+            // add each user to the group
+            for (String groupUser : group.getUsers()) {
+                org.apache.nifi.authorization.file.tenants.generated.Group.User jaxbGroupUser = new org.apache.nifi.authorization.file.tenants.generated.Group.User();
+                jaxbGroupUser.setIdentifier(groupUser);
+                jaxbGroup.getUser().add(jaxbGroupUser);
+            }
 
-        // add each user to the group
-        for (String groupUser : group.getUsers()) {
-            org.apache.nifi.authorization.file.tenants.generated.Group.User jaxbGroupUser = new org.apache.nifi.authorization.file.tenants.generated.Group.User();
-            jaxbGroupUser.setIdentifier(groupUser);
-            jaxbGroup.getUser().add(jaxbGroupUser);
+            tenants.getGroups().getGroup().add(jaxbGroup);
+        }
+
+        final List<User> users = usersAndGroups.getUsers();
+        for (final User user : users) {
+            final org.apache.nifi.authorization.file.tenants.generated.User jaxbUser = createJAXBUser(user);
+            tenants.getUsers().getUser().add(jaxbUser);
         }
 
-        tenants.getGroups().getGroup().add(jaxbGroup);
         saveAndRefreshHolder(tenants);
+    }
 
+    @Override
+    public synchronized Group addGroup(Group group) throws AuthorizationAccessException {
+        if (group == null) {
+            throw new IllegalArgumentException("Group cannot be null");
+        }
+
+        final UsersAndGroups usersAndGroups = new UsersAndGroups(Collections.emptyList(), Collections.singletonList(group));
+        addUsersAndGroups(usersAndGroups);
         return userGroupHolder.get().getGroupsById().get(group.getIdentifier());
     }
 
@@ -455,36 +466,92 @@ public class FileUserGroupProvider implements ConfigurableUserGroupProvider {
     @Override
     public synchronized void inheritFingerprint(String fingerprint) throws AuthorizationAccessException {
         final UsersAndGroups usersAndGroups = parseUsersAndGroups(fingerprint);
-        usersAndGroups.getUsers().forEach(user -> addUser(user));
-        usersAndGroups.getGroups().forEach(group -> addGroup(group));
+        inherit(usersAndGroups);
+    }
+
+    private synchronized void inherit(final UsersAndGroups usersAndGroups) {
+        addUsersAndGroups(usersAndGroups);
+    }
+
+    @Override
+    public synchronized void forciblyInheritFingerprint(final String fingerprint) throws AuthorizationAccessException {
+        if (fingerprint == null || fingerprint.trim().isEmpty()) {
+            logger.info("Inheriting Empty Users & Groups. Will backup existing Uesrs & Groups first.");
+            backupUsersAndGroups();
+            purgeUsersAndGroups();
+
+            return;
+        }
+
+        final UsersAndGroups usersAndGroups = parseUsersAndGroups(fingerprint);
+
+        if (isInheritable(usersAndGroups)) {
+            logger.debug("Inheriting cluster's Users & Groups");
+            inherit(usersAndGroups);
+        } else {
+            logger.info("Cannot directly inherit proposed Users & Groups so will backup existing Users & Groups and then replace with proposed configuration");
+            backupUsersAndGroups();
+            purgeUsersAndGroups();
+            addUsersAndGroups(usersAndGroups);
+        }
+    }
+
+    public void backupUsersAndGroups() throws AuthorizationAccessException {
+        final UserGroupHolder holder = userGroupHolder.get();
+        final Tenants tenants = holder.getTenants();
+
+        final DateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd-HH-mm-ss");
+        final String timestamp = dateFormat.format(new Date());
+        final File destinationFile = new File(tenantsFile.getParentFile(), tenantsFile.getName() + "." + timestamp);
+        logger.info("Writing backup of Users & Groups to {}", destinationFile.getAbsolutePath());
+
+        try {
+            saveTenants(tenants, destinationFile);
+        } catch (final JAXBException jaxb) {
+            throw new AuthorizationAccessException("Could not backup existing Users and Groups so will not inherit new Users and Groups", jaxb);
+        }
+    }
+
+    public synchronized void purgeUsersAndGroups() {
+        final UserGroupHolder holder = userGroupHolder.get();
+        final Tenants tenants = holder.getTenants();
+
+        tenants.setGroups(new Groups());
+        tenants.setUsers(new Users());
+
+        saveAndRefreshHolder(tenants);
     }
 
     @Override
     public void checkInheritability(String proposedFingerprint) throws AuthorizationAccessException {
+        final UsersAndGroups proposedUsersAndGroups;
         try {
             // ensure we understand the proposed fingerprint
-            parseUsersAndGroups(proposedFingerprint);
+            proposedUsersAndGroups = parseUsersAndGroups(proposedFingerprint);
         } catch (final AuthorizationAccessException e) {
             throw new UninheritableAuthorizationsException("Unable to parse the proposed fingerprint: " + e);
         }
 
-        final UserGroupHolder usersAndGroups = userGroupHolder.get();
-
         // ensure we are in a proper state to inherit the fingerprint
-        if (!usersAndGroups.getAllUsers().isEmpty() || !usersAndGroups.getAllGroups().isEmpty()) {
+        if (!isInheritable(proposedUsersAndGroups)) {
             throw new UninheritableAuthorizationsException("Proposed fingerprint is not inheritable because the current users and groups is not empty.");
         }
     }
 
+    private boolean isInheritable(final UsersAndGroups proposedUsersAndGroups) {
+        final UserGroupHolder usersAndGroups = userGroupHolder.get();
+        return usersAndGroups.getAllUsers().isEmpty() && usersAndGroups.getAllGroups().isEmpty();
+    }
+
     @Override
     public String getFingerprint() throws AuthorizationAccessException {
         final UserGroupHolder usersAndGroups = userGroupHolder.get();
 
         final List<User> users = new ArrayList<>(usersAndGroups.getAllUsers());
-        Collections.sort(users, Comparator.comparing(User::getIdentifier));
+        users.sort(Comparator.comparing(User::getIdentifier));
 
         final List<Group> groups = new ArrayList<>(usersAndGroups.getAllGroups());
-        Collections.sort(groups, Comparator.comparing(Group::getIdentifier));
+        groups.sort(Comparator.comparing(Group::getIdentifier));
 
         XMLStreamWriter writer = null;
         final StringWriter out = new StringWriter();
@@ -606,9 +673,8 @@ public class FileUserGroupProvider implements ConfigurableUserGroupProvider {
      *
      * @throws JAXBException            Unable to reload the authorized users file
      * @throws IllegalStateException    Unable to sync file with restore
-     * @throws SAXException             Unable to unmarshall tenants
      */
-    private synchronized void load() throws JAXBException, IllegalStateException, SAXException {
+    private synchronized void load() throws JAXBException, IllegalStateException {
         final Tenants tenants = unmarshallTenants();
         if (tenants.getUsers() == null) {
             tenants.setUsers(new Users());
@@ -637,10 +703,14 @@ public class FileUserGroupProvider implements ConfigurableUserGroupProvider {
     }
 
     private void saveTenants(final Tenants tenants) throws JAXBException {
+        saveTenants(tenants, tenantsFile);
+    }
+
+    private void saveTenants(final Tenants tenants, final File destinationFile) throws JAXBException {
         final Marshaller marshaller = JAXB_TENANTS_CONTEXT.createMarshaller();
         marshaller.setSchema(tenantsSchema);
         marshaller.setProperty(Marshaller.JAXB_FORMATTED_OUTPUT, true);
-        marshaller.marshal(tenants, tenantsFile);
+        marshaller.marshal(tenants, destinationFile);
     }
 
     private Tenants unmarshallTenants() throws JAXBException {
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/EncryptedRepositoryRecordSerdeFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/EncryptedRepositoryRecordSerdeFactory.java
index 0cc2d72..5b28177 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/EncryptedRepositoryRecordSerdeFactory.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/EncryptedRepositoryRecordSerdeFactory.java
@@ -48,11 +48,11 @@ public class EncryptedRepositoryRecordSerdeFactory extends StandardRepositoryRec
     }
 
     @Override
-    public SerDe<RepositoryRecord> createSerDe(String encodingName) {
+    public SerDe<SerializedRepositoryRecord> createSerDe(String encodingName) {
         // If no encoding is provided, use the encrypted as the default
         if (encodingName == null || EncryptedSchemaRepositoryRecordSerde.class.getName().equals(encodingName)) {
             // Delegate the creation of the wrapped serde to the standard factory
-            final SerDe<RepositoryRecord> serde = super.createSerDe(null);
+            final SerDe<SerializedRepositoryRecord> serde = super.createSerDe(null);
 
             // Retrieve encryption configuration
             try {
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/EncryptedSchemaRepositoryRecordSerde.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/EncryptedSchemaRepositoryRecordSerde.java
index f6daa4b..e71af0c 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/EncryptedSchemaRepositoryRecordSerde.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/EncryptedSchemaRepositoryRecordSerde.java
@@ -48,9 +48,9 @@ import org.wali.UpdateType;
  * <a href="https://nifi.apache.org/docs/nifi-docs/html/administration-guide.html#encrypted-flowfile-repository-properties">Apache NiFi Admin Guide - Encrypted FlowFile
  * Repository Properties</a>.
  */
-public class EncryptedSchemaRepositoryRecordSerde implements SerDe<RepositoryRecord> {
+public class EncryptedSchemaRepositoryRecordSerde implements SerDe<SerializedRepositoryRecord> {
     private static final Logger logger = LoggerFactory.getLogger(EncryptedSchemaRepositoryRecordSerde.class);
-    private final SerDe<RepositoryRecord> wrappedSerDe;
+    private final SerDe<SerializedRepositoryRecord> wrappedSerDe;
     private final KeyProvider keyProvider;
     private String activeKeyId;
 
@@ -61,7 +61,7 @@ public class EncryptedSchemaRepositoryRecordSerde implements SerDe<RepositoryRec
      * @param flowFileRepositoryEncryptionConfiguration the configuration values necessary to encrypt/decrypt the data
      * @throws IOException if there is a problem retrieving the configuration values
      */
-    public EncryptedSchemaRepositoryRecordSerde(final SerDe<RepositoryRecord> wrappedSerDe, final FlowFileRepositoryEncryptionConfiguration
+    public EncryptedSchemaRepositoryRecordSerde(final SerDe<SerializedRepositoryRecord> wrappedSerDe, final FlowFileRepositoryEncryptionConfiguration
             flowFileRepositoryEncryptionConfiguration) throws IOException {
         if (wrappedSerDe == null) {
             throw new IllegalArgumentException("This implementation must be provided another serde instance to function");
@@ -82,7 +82,7 @@ public class EncryptedSchemaRepositoryRecordSerde implements SerDe<RepositoryRec
      * @param niFiProperties the configuration values necessary to encrypt/decrypt the data
      * @throws IOException if there is a problem retrieving the configuration values
      */
-    public EncryptedSchemaRepositoryRecordSerde(final SerDe<RepositoryRecord> wrappedSerDe, final NiFiProperties niFiProperties) throws IOException {
+    public EncryptedSchemaRepositoryRecordSerde(final SerDe<SerializedRepositoryRecord> wrappedSerDe, final NiFiProperties niFiProperties) throws IOException {
         this(wrappedSerDe, new FlowFileRepositoryEncryptionConfiguration(niFiProperties));
     }
 
@@ -134,11 +134,11 @@ public class EncryptedSchemaRepositoryRecordSerde implements SerDe<RepositoryRec
      * @param out                 stream to write to
      * @throws IOException if fail during write
      * @deprecated it is not beneficial to serialize the deltas, so this method just passes through to
-     * {@link #serializeRecord(RepositoryRecord, DataOutputStream)}. It is preferable to use that method directly.
+     * {@link #serializeRecord(SerializedRepositoryRecord, DataOutputStream)}. It is preferable to use that method directly.
      */
     @Deprecated
     @Override
-    public void serializeEdit(RepositoryRecord previousRecordState, RepositoryRecord newRecordState, DataOutputStream out) throws IOException {
+    public void serializeEdit(SerializedRepositoryRecord previousRecordState, SerializedRepositoryRecord newRecordState, DataOutputStream out) throws IOException {
         serializeRecord(newRecordState, out);
     }
 
@@ -150,7 +150,7 @@ public class EncryptedSchemaRepositoryRecordSerde implements SerDe<RepositoryRec
      * @throws IOException if there is a problem writing to the stream
      */
     @Override
-    public void serializeRecord(final RepositoryRecord record, final DataOutputStream out) throws IOException {
+    public void serializeRecord(final SerializedRepositoryRecord record, final DataOutputStream out) throws IOException {
         // Create BAOS wrapped in DOS to intercept the output
         ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
         DataOutputStream tempDataStream = new DataOutputStream(byteArrayOutputStream);
@@ -222,7 +222,7 @@ public class EncryptedSchemaRepositoryRecordSerde implements SerDe<RepositoryRec
      */
     @Deprecated
     @Override
-    public RepositoryRecord deserializeEdit(DataInputStream in, Map<Object, RepositoryRecord> currentRecordStates, int version) throws IOException {
+    public SerializedRepositoryRecord deserializeEdit(DataInputStream in, Map<Object, SerializedRepositoryRecord> currentRecordStates, int version) throws IOException {
         return deserializeRecord(in, version);
 
         // deserializeRecord may return a null if there is no more data. However, when we are deserializing
@@ -242,7 +242,7 @@ public class EncryptedSchemaRepositoryRecordSerde implements SerDe<RepositoryRec
      * @throws IOException if there is a problem reading from the stream
      */
     @Override
-    public RepositoryRecord deserializeRecord(final DataInputStream in, final int version) throws IOException {
+    public SerializedRepositoryRecord deserializeRecord(final DataInputStream in, final int version) throws IOException {
         // Read the expected length of the encrypted record (including the encryption metadata)
         int encryptedRecordLength = in.readInt();
         if (encryptedRecordLength == -1) {
@@ -258,7 +258,7 @@ public class EncryptedSchemaRepositoryRecordSerde implements SerDe<RepositoryRec
         DataInputStream wrappedInputStream = decryptToStream(cipherBytes);
 
         // Deserialize the plain bytes using the delegate serde
-        final RepositoryRecord deserializedRecord = wrappedSerDe.deserializeRecord(wrappedInputStream, version);
+        final SerializedRepositoryRecord deserializedRecord = wrappedSerDe.deserializeRecord(wrappedInputStream, version);
         logger.debug("Deserialized flowfile record {} from temp stream", getRecordIdentifier(deserializedRecord));
         return deserializedRecord;
     }
@@ -298,7 +298,7 @@ public class EncryptedSchemaRepositoryRecordSerde implements SerDe<RepositoryRec
      * @return identifier of record
      */
     @Override
-    public Object getRecordIdentifier(RepositoryRecord record) {
+    public Object getRecordIdentifier(SerializedRepositoryRecord record) {
         return wrappedSerDe.getRecordIdentifier(record);
     }
 
@@ -309,7 +309,7 @@ public class EncryptedSchemaRepositoryRecordSerde implements SerDe<RepositoryRec
      * @return update type
      */
     @Override
-    public UpdateType getUpdateType(RepositoryRecord record) {
+    public UpdateType getUpdateType(SerializedRepositoryRecord record) {
         return wrappedSerDe.getUpdateType(record);
     }
 
@@ -326,7 +326,7 @@ public class EncryptedSchemaRepositoryRecordSerde implements SerDe<RepositoryRec
      * @return location
      */
     @Override
-    public String getLocation(RepositoryRecord record) {
+    public String getLocation(SerializedRepositoryRecord record) {
         return wrappedSerDe.getLocation(record);
     }
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/ReconstitutedSerializedRepositoryRecord.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/ReconstitutedSerializedRepositoryRecord.java
new file mode 100644
index 0000000..bda7458
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/ReconstitutedSerializedRepositoryRecord.java
@@ -0,0 +1,111 @@
+/*
+ * 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.repository;
+
+import org.apache.nifi.controller.repository.claim.ContentClaim;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+
+public class ReconstitutedSerializedRepositoryRecord implements SerializedRepositoryRecord {
+    private final String queueIdentifier;
+    private final RepositoryRecordType type;
+    private final FlowFileRecord flowFile;
+    private final String swapLocation;
+
+    private ReconstitutedSerializedRepositoryRecord(final Builder builder) {
+        this.queueIdentifier = builder.queueIdentifier;
+        this.type = builder.type;
+        this.flowFile = builder.flowFile;
+        this.swapLocation = builder.swapLocation;
+    }
+
+    @Override
+    public String getQueueIdentifier() {
+        return queueIdentifier;
+    }
+
+    @Override
+    public RepositoryRecordType getType() {
+        return type;
+    }
+
+    @Override
+    public ContentClaim getContentClaim() {
+        return flowFile.getContentClaim();
+    }
+
+    @Override
+    public long getClaimOffset() {
+        return flowFile.getContentClaimOffset();
+    }
+
+    @Override
+    public FlowFileRecord getFlowFileRecord() {
+        return flowFile;
+    }
+
+    @Override
+    public boolean isMarkedForAbort() {
+        return false;
+    }
+
+    @Override
+    public boolean isAttributesChanged() {
+        return false;
+    }
+
+    @Override
+    public String getSwapLocation() {
+        return swapLocation;
+    }
+
+    @Override
+    public String toString() {
+        return "ReconstitutedSerializedRepositoryRecord[recordType=" + type + ", queueId=" + queueIdentifier + ", flowFileUuid=" + flowFile.getAttribute(CoreAttributes.UUID.key())
+            + ", attributesChanged=" + isAttributesChanged() + "]";
+    }
+
+    public static class Builder {
+        private String queueIdentifier;
+        private RepositoryRecordType type;
+        private FlowFileRecord flowFile;
+        private String swapLocation;
+
+        public Builder queueIdentifier(final String queueIdentifier) {
+            this.queueIdentifier = queueIdentifier;
+            return this;
+        }
+
+        public Builder type(final RepositoryRecordType type) {
+            this.type = type;
+            return this;
+        }
+
+        public Builder flowFileRecord(final FlowFileRecord flowFileRecord) {
+            this.flowFile = flowFileRecord;
+            return this;
+        }
+
+        public Builder swapLocation(final String swapLocation) {
+            this.swapLocation = swapLocation;
+            return this;
+        }
+
+        public ReconstitutedSerializedRepositoryRecord build() {
+            return new ReconstitutedSerializedRepositoryRecord(this);
+        }
+    }
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/RepositoryRecordSerde.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/RepositoryRecordSerde.java
index 44ed62d..56328c6 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/RepositoryRecordSerde.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/RepositoryRecordSerde.java
@@ -17,34 +17,18 @@
 
 package org.apache.nifi.controller.repository;
 
-import java.util.Map;
-
-import org.apache.nifi.controller.queue.FlowFileQueue;
 import org.wali.SerDe;
 import org.wali.UpdateType;
 
-public abstract class RepositoryRecordSerde implements SerDe<RepositoryRecord> {
-    private Map<String, FlowFileQueue> flowFileQueueMap = null;
-
-    protected void setQueueMap(final Map<String, FlowFileQueue> queueMap) {
-        this.flowFileQueueMap = queueMap;
-    }
-
-    protected Map<String, FlowFileQueue> getQueueMap() {
-        return flowFileQueueMap;
-    }
-
-    protected FlowFileQueue getFlowFileQueue(final String queueId) {
-        return flowFileQueueMap.get(queueId);
-    }
+public abstract class RepositoryRecordSerde implements SerDe<SerializedRepositoryRecord> {
 
     @Override
-    public Long getRecordIdentifier(final RepositoryRecord record) {
-        return record.getCurrent().getId();
+    public Long getRecordIdentifier(final SerializedRepositoryRecord record) {
+        return record.getFlowFileRecord().getId();
     }
 
     @Override
-    public UpdateType getUpdateType(final RepositoryRecord record) {
+    public UpdateType getUpdateType(final SerializedRepositoryRecord record) {
         switch (record.getType()) {
             case CONTENTMISSING:
             case DELETE:
@@ -62,7 +46,7 @@ public abstract class RepositoryRecordSerde implements SerDe<RepositoryRecord> {
     }
 
     @Override
-    public String getLocation(final RepositoryRecord record) {
+    public String getLocation(final SerializedRepositoryRecord record) {
         return record.getSwapLocation();
     }
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/RepositoryRecordSerdeFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/RepositoryRecordSerdeFactory.java
index 38b7845..c0886cd 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/RepositoryRecordSerdeFactory.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/RepositoryRecordSerdeFactory.java
@@ -16,13 +16,8 @@
  */
 package org.apache.nifi.controller.repository;
 
-import org.apache.nifi.controller.queue.FlowFileQueue;
 import org.wali.SerDeFactory;
 
-import java.util.Map;
-
-public interface RepositoryRecordSerdeFactory extends SerDeFactory<RepositoryRecord> {
-    void setQueueMap(Map<String, FlowFileQueue> queueMap);
-
-    Long getRecordIdentifier(RepositoryRecord record);
+public interface RepositoryRecordSerdeFactory extends SerDeFactory<SerializedRepositoryRecord> {
+    Long getRecordIdentifier(SerializedRepositoryRecord record);
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/SchemaRepositoryRecordSerde.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/SchemaRepositoryRecordSerde.java
index 0013846..ee87f1b 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/SchemaRepositoryRecordSerde.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/SchemaRepositoryRecordSerde.java
@@ -17,7 +17,6 @@
 
 package org.apache.nifi.controller.repository;
 
-import org.apache.nifi.controller.queue.FlowFileQueue;
 import org.apache.nifi.controller.repository.claim.ContentClaim;
 import org.apache.nifi.controller.repository.claim.ResourceClaimManager;
 import org.apache.nifi.controller.repository.schema.ContentClaimFieldMap;
@@ -34,8 +33,6 @@ import org.apache.nifi.repository.schema.Repetition;
 import org.apache.nifi.repository.schema.SchemaRecordReader;
 import org.apache.nifi.repository.schema.SchemaRecordWriter;
 import org.apache.nifi.repository.schema.SimpleRecordField;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.wali.SerDe;
 
 import java.io.DataInputStream;
@@ -45,8 +42,7 @@ import java.io.File;
 import java.io.IOException;
 import java.util.Map;
 
-public class SchemaRepositoryRecordSerde extends RepositoryRecordSerde implements SerDe<RepositoryRecord> {
-    private static final Logger logger = LoggerFactory.getLogger(SchemaRepositoryRecordSerde.class);
+public class SchemaRepositoryRecordSerde extends RepositoryRecordSerde implements SerDe<SerializedRepositoryRecord> {
     private static final int MAX_ENCODING_VERSION = 2;
 
     private final RecordSchema writeSchema = RepositoryRecordSchema.REPOSITORY_RECORD_SCHEMA_V2;
@@ -66,12 +62,12 @@ public class SchemaRepositoryRecordSerde extends RepositoryRecordSerde implement
     }
 
     @Override
-    public void serializeEdit(final RepositoryRecord previousRecordState, final RepositoryRecord newRecordState, final DataOutputStream out) throws IOException {
+    public void serializeEdit(final SerializedRepositoryRecord previousRecordState, final SerializedRepositoryRecord newRecordState, final DataOutputStream out) throws IOException {
         serializeRecord(newRecordState, out);
     }
 
     @Override
-    public void serializeRecord(final RepositoryRecord record, final DataOutputStream out) throws IOException {
+    public void serializeRecord(final SerializedRepositoryRecord record, final DataOutputStream out) throws IOException {
         final RecordSchema schema;
         switch (record.getType()) {
             case CREATE:
@@ -96,7 +92,7 @@ public class SchemaRepositoryRecordSerde extends RepositoryRecordSerde implement
     }
 
 
-    protected void serializeRecord(final RepositoryRecord record, final DataOutputStream out, RecordSchema schema, RecordSchema repositoryRecordSchema) throws IOException {
+    protected void serializeRecord(final SerializedRepositoryRecord record, final DataOutputStream out, RecordSchema schema, RecordSchema repositoryRecordSchema) throws IOException {
         final RepositoryRecordFieldMap fieldMap = new RepositoryRecordFieldMap(record, schema, contentClaimSchema);
         final RepositoryRecordUpdate update = new RepositoryRecordUpdate(fieldMap, repositoryRecordSchema);
         new SchemaRecordWriter().writeRecord(update, out);
@@ -109,8 +105,8 @@ public class SchemaRepositoryRecordSerde extends RepositoryRecordSerde implement
     }
 
     @Override
-    public RepositoryRecord deserializeEdit(final DataInputStream in, final Map<Object, RepositoryRecord> currentRecordStates, final int version) throws IOException {
-        final RepositoryRecord record = deserializeRecord(in, version);
+    public SerializedRepositoryRecord deserializeEdit(final DataInputStream in, final Map<Object, SerializedRepositoryRecord> currentRecordStates, final int version) throws IOException {
+        final SerializedRepositoryRecord record = deserializeRecord(in, version);
         if (record != null) {
             return record;
         }
@@ -123,9 +119,9 @@ public class SchemaRepositoryRecordSerde extends RepositoryRecordSerde implement
     }
 
     @Override
-    public RepositoryRecord deserializeRecord(final DataInputStream in, final int version) throws IOException {
+    public SerializedRepositoryRecord deserializeRecord(final DataInputStream in, final int version) throws IOException {
         if (recordIterator != null) {
-            final RepositoryRecord record = nextRecord();
+            final SerializedRepositoryRecord record = nextRecord();
             if (record != null) {
                 return record;
             }
@@ -141,7 +137,7 @@ public class SchemaRepositoryRecordSerde extends RepositoryRecordSerde implement
         return nextRecord();
     }
 
-    private RepositoryRecord nextRecord() throws IOException {
+    private SerializedRepositoryRecord nextRecord() throws IOException {
         final Record record;
         try {
             record = recordIterator.next();
@@ -158,7 +154,7 @@ public class SchemaRepositoryRecordSerde extends RepositoryRecordSerde implement
         return createRepositoryRecord(record);
     }
 
-    private RepositoryRecord createRepositoryRecord(final Record updateRecord) throws IOException {
+    private SerializedRepositoryRecord createRepositoryRecord(final Record updateRecord) throws IOException {
         if (updateRecord == null) {
             // null may be returned by reader.readRecord() if it encounters end-of-stream
             return null;
@@ -172,7 +168,7 @@ public class SchemaRepositoryRecordSerde extends RepositoryRecordSerde implement
         final RepositoryRecordType recordType = RepositoryRecordType.valueOf(actionType);
         switch (recordType) {
             case CREATE:
-                return createRecord(record);
+                return createRecord(record, RepositoryRecordType.CREATE, null);
             case CONTENTMISSING:
             case DELETE:
                 return deleteRecord(record);
@@ -189,7 +185,7 @@ public class SchemaRepositoryRecordSerde extends RepositoryRecordSerde implement
 
 
     @SuppressWarnings("unchecked")
-    private StandardRepositoryRecord createRecord(final Record record) {
+    private SerializedRepositoryRecord createRecord(final Record record, final RepositoryRecordType type, final String swapLocation) {
         final StandardFlowFileRecord.Builder ffBuilder = new StandardFlowFileRecord.Builder();
         ffBuilder.id((Long) record.getFieldValue(RepositoryRecordSchema.RECORD_ID));
         ffBuilder.entryDate((Long) record.getFieldValue(FlowFileSchema.ENTRY_DATE));
@@ -210,23 +206,16 @@ public class SchemaRepositoryRecordSerde extends RepositoryRecordSerde implement
         final FlowFileRecord flowFileRecord = ffBuilder.build();
 
         final String queueId = (String) record.getFieldValue(RepositoryRecordSchema.QUEUE_IDENTIFIER);
-        final FlowFileQueue queue = getFlowFileQueue(queueId);
+        final SerializedRepositoryRecord repoRecord = new ReconstitutedSerializedRepositoryRecord.Builder()
+            .flowFileRecord(flowFileRecord)
+            .queueIdentifier(queueId)
+            .type(type)
+            .swapLocation(swapLocation)
+            .build();
 
-        final StandardRepositoryRecord repoRecord = new StandardRepositoryRecord(queue, flowFileRecord);
-        requireFlowFileQueue(repoRecord, queueId);
         return repoRecord;
     }
 
-    private void requireFlowFileQueue(final StandardRepositoryRecord repoRecord, final String queueId) {
-        if (queueId == null || queueId.trim().isEmpty()) {
-            logger.warn("{} does not have a Queue associated with it; this record will be discarded", repoRecord.getCurrent());
-            repoRecord.markForAbort();
-        } else if (repoRecord.getOriginalQueue() == null) {
-            logger.warn("{} maps to unknown Queue {}; this record will be discarded", repoRecord.getCurrent(), queueId);
-            repoRecord.markForAbort();
-        }
-    }
-
     private void populateContentClaim(final StandardFlowFileRecord.Builder ffBuilder, final Record record) {
         final Object claimMap = record.getFieldValue(FlowFileSchema.CONTENT_CLAIM);
         if (claimMap == null) {
@@ -241,41 +230,44 @@ public class SchemaRepositoryRecordSerde extends RepositoryRecordSerde implement
         ffBuilder.contentClaimOffset(offset);
     }
 
-    private RepositoryRecord updateRecord(final Record record) {
-        return createRecord(record);
+    private SerializedRepositoryRecord updateRecord(final Record record) {
+        return createRecord(record, RepositoryRecordType.UPDATE, null);
     }
 
-    private RepositoryRecord deleteRecord(final Record record) {
+    private SerializedRepositoryRecord deleteRecord(final Record record) {
         final Long recordId = (Long) record.getFieldValue(RepositoryRecordSchema.RECORD_ID_FIELD);
         final StandardFlowFileRecord.Builder ffBuilder = new StandardFlowFileRecord.Builder().id(recordId);
         final FlowFileRecord flowFileRecord = ffBuilder.build();
 
-        final StandardRepositoryRecord repoRecord = new StandardRepositoryRecord((FlowFileQueue) null, flowFileRecord);
-        repoRecord.markForDelete();
+        final SerializedRepositoryRecord repoRecord = new ReconstitutedSerializedRepositoryRecord.Builder()
+            .flowFileRecord(flowFileRecord)
+            .type(RepositoryRecordType.DELETE)
+            .build();
+
         return repoRecord;
     }
 
-    private RepositoryRecord swapInRecord(final Record record) {
-        final StandardRepositoryRecord repoRecord = createRecord(record);
+    private SerializedRepositoryRecord swapInRecord(final Record record) {
         final String swapLocation = (String) record.getFieldValue(new SimpleRecordField(RepositoryRecordSchema.SWAP_LOCATION, FieldType.STRING, Repetition.EXACTLY_ONE));
-        repoRecord.setSwapLocation(swapLocation);
-
-        final String queueId = (String) record.getFieldValue(RepositoryRecordSchema.QUEUE_IDENTIFIER);
-        requireFlowFileQueue(repoRecord, queueId);
+        final SerializedRepositoryRecord repoRecord = createRecord(record, RepositoryRecordType.SWAP_IN, swapLocation);
         return repoRecord;
     }
 
-    private RepositoryRecord swapOutRecord(final Record record) {
+    private SerializedRepositoryRecord swapOutRecord(final Record record) {
         final Long recordId = (Long) record.getFieldValue(RepositoryRecordSchema.RECORD_ID_FIELD);
         final String queueId = (String) record.getFieldValue(new SimpleRecordField(RepositoryRecordSchema.QUEUE_IDENTIFIER, FieldType.STRING, Repetition.EXACTLY_ONE));
         final String swapLocation = (String) record.getFieldValue(new SimpleRecordField(RepositoryRecordSchema.SWAP_LOCATION, FieldType.STRING, Repetition.EXACTLY_ONE));
-        final FlowFileQueue queue = getFlowFileQueue(queueId);
 
         final FlowFileRecord flowFileRecord = new StandardFlowFileRecord.Builder()
             .id(recordId)
             .build();
 
-        return new StandardRepositoryRecord(queue, flowFileRecord, swapLocation);
+        return new ReconstitutedSerializedRepositoryRecord.Builder()
+            .flowFileRecord(flowFileRecord)
+            .type(RepositoryRecordType.SWAP_OUT)
+            .swapLocation(swapLocation)
+            .queueIdentifier(queueId)
+            .build();
     }
 
     @Override
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/SerializedRepositoryRecord.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/SerializedRepositoryRecord.java
new file mode 100644
index 0000000..51eb034
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/SerializedRepositoryRecord.java
@@ -0,0 +1,62 @@
+/*
+ * 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.repository;
+
+import org.apache.nifi.controller.repository.claim.ContentClaim;
+
+public interface SerializedRepositoryRecord {
+
+    /**
+     * @return the ID of the FlowFile Queue that this Record belongs to
+     */
+    String getQueueIdentifier();
+
+    /**
+     * @return type of update that this record encapsulates
+     */
+    RepositoryRecordType getType();
+
+    /**
+     * @return ContentClaim for the FlowFile
+     */
+    ContentClaim getContentClaim();
+
+    /**
+     * @return byte offset into the Content Claim where this FlowFile's content begins
+     */
+    long getClaimOffset();
+
+    /**
+     * @return the swap location for swap in/swap out records
+     */
+    String getSwapLocation();
+
+    /**
+     * @return FlowFile being encapsulated by this record
+     */
+    FlowFileRecord getFlowFileRecord();
+
+    /**
+     * @return whether or not the record is marked for abort
+     */
+    boolean isMarkedForAbort();
+
+    /**
+     * @return whether or not attributes have changed since the record was last persisted
+     */
+    boolean isAttributesChanged();
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/StandardRepositoryRecordSerdeFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/StandardRepositoryRecordSerdeFactory.java
index 549ba87..ac1989c 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/StandardRepositoryRecordSerdeFactory.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/StandardRepositoryRecordSerdeFactory.java
@@ -17,59 +17,41 @@
 
 package org.apache.nifi.controller.repository;
 
-import java.util.Map;
-import org.apache.nifi.controller.queue.FlowFileQueue;
 import org.apache.nifi.controller.repository.claim.ResourceClaimManager;
 import org.wali.SerDe;
 import org.wali.UpdateType;
 
 public class StandardRepositoryRecordSerdeFactory implements RepositoryRecordSerdeFactory {
-    private final String LEGACY_SERDE_ENCODING_NAME = "org.apache.nifi.controller.repository.WriteAheadFlowFileRepository$WriteAheadRecordSerde";
+    private static final String LEGACY_SERDE_ENCODING_NAME = "org.apache.nifi.controller.repository.WriteAheadFlowFileRepository$WriteAheadRecordSerde";
     private final ResourceClaimManager resourceClaimManager;
-    private Map<String, FlowFileQueue> flowFileQueueMap = null;
 
     public StandardRepositoryRecordSerdeFactory(final ResourceClaimManager claimManager) {
         this.resourceClaimManager = claimManager;
     }
 
     @Override
-    public void setQueueMap(final Map<String, FlowFileQueue> queueMap) {
-        this.flowFileQueueMap = queueMap;
-    }
-
-    protected Map<String, FlowFileQueue> getQueueMap() {
-        return flowFileQueueMap;
-    }
-
-    @Override
-    public SerDe<RepositoryRecord> createSerDe(final String encodingName) {
+    public SerDe<SerializedRepositoryRecord> createSerDe(final String encodingName) {
         if (encodingName == null || SchemaRepositoryRecordSerde.class.getName().equals(encodingName)) {
             final SchemaRepositoryRecordSerde serde = new SchemaRepositoryRecordSerde(resourceClaimManager);
-            serde.setQueueMap(flowFileQueueMap);
             return serde;
         }
 
         if (WriteAheadRepositoryRecordSerde.class.getName().equals(encodingName)
             || LEGACY_SERDE_ENCODING_NAME.equals(encodingName)) {
             final WriteAheadRepositoryRecordSerde serde = new WriteAheadRepositoryRecordSerde(resourceClaimManager);
-            serde.setQueueMap(flowFileQueueMap);
             return serde;
         }
 
         throw new IllegalArgumentException("Cannot create Deserializer for Repository Records because the encoding '" + encodingName + "' is not known");
     }
 
-    protected FlowFileQueue getFlowFileQueue(final String queueId) {
-        return flowFileQueueMap.get(queueId);
-    }
-
     @Override
-    public Long getRecordIdentifier(final RepositoryRecord record) {
-        return record.getCurrent().getId();
+    public Long getRecordIdentifier(final SerializedRepositoryRecord record) {
+        return record.getFlowFileRecord().getId();
     }
 
     @Override
-    public UpdateType getUpdateType(final RepositoryRecord record) {
+    public UpdateType getUpdateType(final SerializedRepositoryRecord record) {
         switch (record.getType()) {
             case CONTENTMISSING:
             case DELETE:
@@ -87,7 +69,7 @@ public class StandardRepositoryRecordSerdeFactory implements RepositoryRecordSer
     }
 
     @Override
-    public String getLocation(final RepositoryRecord record) {
+    public String getLocation(final SerializedRepositoryRecord record) {
         return record.getSwapLocation();
     }
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/WriteAheadRepositoryRecordSerde.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/WriteAheadRepositoryRecordSerde.java
index 4155c44..4c292f7 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/WriteAheadRepositoryRecordSerde.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/WriteAheadRepositoryRecordSerde.java
@@ -17,16 +17,6 @@
 
 package org.apache.nifi.controller.repository;
 
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.EOFException;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.nio.charset.StandardCharsets;
-import java.util.HashMap;
-import java.util.Map;
-import org.apache.nifi.controller.queue.FlowFileQueue;
 import org.apache.nifi.controller.repository.claim.ContentClaim;
 import org.apache.nifi.controller.repository.claim.ResourceClaim;
 import org.apache.nifi.controller.repository.claim.ResourceClaimManager;
@@ -37,7 +27,17 @@ import org.slf4j.LoggerFactory;
 import org.wali.SerDe;
 import org.wali.UpdateType;
 
-public class WriteAheadRepositoryRecordSerde extends RepositoryRecordSerde implements SerDe<RepositoryRecord> {
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.charset.StandardCharsets;
+import java.util.HashMap;
+import java.util.Map;
+
+public class WriteAheadRepositoryRecordSerde extends RepositoryRecordSerde implements SerDe<SerializedRepositoryRecord> {
     private static final Logger logger = LoggerFactory.getLogger(WriteAheadRepositoryRecordSerde.class);
 
     private static final int CURRENT_ENCODING_VERSION = 9;
@@ -56,16 +56,17 @@ public class WriteAheadRepositoryRecordSerde extends RepositoryRecordSerde imple
     }
 
     @Override
-    public void serializeEdit(final RepositoryRecord previousRecordState, final RepositoryRecord record, final DataOutputStream out) throws IOException {
+    public void serializeEdit(final SerializedRepositoryRecord previousRecordState, final SerializedRepositoryRecord record, final DataOutputStream out) throws IOException {
         serializeEdit(previousRecordState, record, out, false);
     }
 
-    public void serializeEdit(final RepositoryRecord previousRecordState, final RepositoryRecord record, final DataOutputStream out, final boolean forceAttributesWritten) throws IOException {
+    public void serializeEdit(final SerializedRepositoryRecord previousRecordState, final SerializedRepositoryRecord record, final DataOutputStream out, final boolean forceAttributesWritten)
+            throws IOException {
         if (record.isMarkedForAbort()) {
             logger.warn("Repository Record {} is marked to be aborted; it will be persisted in the FlowFileRepository as a DELETE record", record);
             out.write(ACTION_DELETE);
             out.writeLong(getRecordIdentifier(record));
-            serializeContentClaim(record.getCurrentClaim(), record.getCurrentClaimOffset(), out);
+            serializeContentClaim(record.getContentClaim(), record.getClaimOffset(), out);
             return;
         }
 
@@ -74,7 +75,7 @@ public class WriteAheadRepositoryRecordSerde extends RepositoryRecordSerde imple
         if (updateType.equals(UpdateType.DELETE)) {
             out.write(ACTION_DELETE);
             out.writeLong(getRecordIdentifier(record));
-            serializeContentClaim(record.getCurrentClaim(), record.getCurrentClaimOffset(), out);
+            serializeContentClaim(record.getContentClaim(), record.getClaimOffset(), out);
             return;
         }
 
@@ -82,21 +83,18 @@ public class WriteAheadRepositoryRecordSerde extends RepositoryRecordSerde imple
         // However, on restart, we will restore the FlowFile and set this connection to its "originalConnection".
         // If we then serialize the FlowFile again before it's transferred, it's important to allow this to happen,
         // so we use the originalConnection instead
-        FlowFileQueue associatedQueue = record.getDestination();
-        if (associatedQueue == null) {
-            associatedQueue = record.getOriginalQueue();
-        }
+        final String associatedQueueId = record.getQueueIdentifier();
 
         if (updateType.equals(UpdateType.SWAP_OUT)) {
             out.write(ACTION_SWAPPED_OUT);
             out.writeLong(getRecordIdentifier(record));
-            out.writeUTF(associatedQueue.getIdentifier());
+            out.writeUTF(associatedQueueId);
             out.writeUTF(getLocation(record));
             return;
         }
 
-        final FlowFile flowFile = record.getCurrent();
-        final ContentClaim claim = record.getCurrentClaim();
+        final FlowFile flowFile = record.getFlowFileRecord();
+        final ContentClaim claim = record.getContentClaim();
 
         switch (updateType) {
             case UPDATE:
@@ -122,15 +120,15 @@ public class WriteAheadRepositoryRecordSerde extends RepositoryRecordSerde imple
         out.writeLong(flowFile.getQueueDateIndex());
         out.writeLong(flowFile.getSize());
 
-        if (associatedQueue == null) {
+        if (associatedQueueId == null) {
             logger.warn("{} Repository Record {} has no Connection associated with it; it will be destroyed on restart",
                 new Object[] {this, record});
             writeString("", out);
         } else {
-            writeString(associatedQueue.getIdentifier(), out);
+            writeString(associatedQueueId, out);
         }
 
-        serializeContentClaim(claim, record.getCurrentClaimOffset(), out);
+        serializeContentClaim(claim, record.getClaimOffset(), out);
 
         if (forceAttributesWritten || record.isAttributesChanged() || updateType == UpdateType.CREATE || updateType == UpdateType.SWAP_IN) {
             out.write(1);   // indicate attributes changed
@@ -150,7 +148,7 @@ public class WriteAheadRepositoryRecordSerde extends RepositoryRecordSerde imple
     }
 
     @Override
-    public RepositoryRecord deserializeEdit(final DataInputStream in, final Map<Object, RepositoryRecord> currentRecordStates, final int version) throws IOException {
+    public SerializedRepositoryRecord deserializeEdit(final DataInputStream in, final Map<Object, SerializedRepositoryRecord> currentRecordStates, final int version) throws IOException {
         final int action = in.read();
         final long recordId = in.readLong();
         if (action == ACTION_DELETE) {
@@ -161,8 +159,10 @@ public class WriteAheadRepositoryRecordSerde extends RepositoryRecordSerde imple
             }
 
             final FlowFileRecord flowFileRecord = ffBuilder.build();
-            final StandardRepositoryRecord record = new StandardRepositoryRecord(null, flowFileRecord);
-            record.markForDelete();
+            final SerializedRepositoryRecord record = new ReconstitutedSerializedRepositoryRecord.Builder()
+                .type(RepositoryRecordType.DELETE)
+                .flowFileRecord(flowFileRecord)
+                .build();
 
             return record;
         }
@@ -170,20 +170,27 @@ public class WriteAheadRepositoryRecordSerde extends RepositoryRecordSerde imple
         if (action == ACTION_SWAPPED_OUT) {
             final String queueId = in.readUTF();
             final String location = in.readUTF();
-            final FlowFileQueue queue = getFlowFileQueue(queueId);
 
             final FlowFileRecord flowFileRecord = new StandardFlowFileRecord.Builder()
                 .id(recordId)
                 .build();
 
-            return new StandardRepositoryRecord(queue, flowFileRecord, location);
+
+            final SerializedRepositoryRecord record = new ReconstitutedSerializedRepositoryRecord.Builder()
+                .type(RepositoryRecordType.SWAP_OUT)
+                .queueIdentifier(queueId)
+                .swapLocation(location)
+                .flowFileRecord(flowFileRecord)
+                .build();
+
+            return record;
         }
 
         final StandardFlowFileRecord.Builder ffBuilder = new StandardFlowFileRecord.Builder();
-        final RepositoryRecord record = currentRecordStates.get(recordId);
+        final SerializedRepositoryRecord record = currentRecordStates.get(recordId);
         ffBuilder.id(recordId);
         if (record != null) {
-            ffBuilder.fromFlowFile(record.getCurrent());
+            ffBuilder.fromFlowFile(record.getFlowFileRecord());
         }
         ffBuilder.entryDate(in.readLong());
 
@@ -249,26 +256,21 @@ public class WriteAheadRepositoryRecordSerde extends RepositoryRecordSerde imple
             swapLocation = in.readUTF();
         }
 
-        final FlowFileQueue queue = getFlowFileQueue(connectionId);
-        final StandardRepositoryRecord standardRepoRecord = new StandardRepositoryRecord(queue, flowFile);
-        if (swapLocation != null) {
-            standardRepoRecord.setSwapLocation(swapLocation);
-        }
+        final RepositoryRecordType recordType = getRecordType(action);
 
-        if (connectionId.isEmpty()) {
-            logger.warn("{} does not have a Queue associated with it; this record will be discarded", flowFile);
-            standardRepoRecord.markForAbort();
-        } else if (queue == null) {
-            logger.warn("{} maps to unknown Queue {}; this record will be discarded", flowFile, connectionId);
-            standardRepoRecord.markForAbort();
-        }
+        final SerializedRepositoryRecord repositoryRecord = new ReconstitutedSerializedRepositoryRecord.Builder()
+            .flowFileRecord(flowFile)
+            .queueIdentifier(connectionId)
+            .swapLocation(swapLocation)
+            .type(recordType)
+            .build();
 
         recordsRestored++;
-        return standardRepoRecord;
+        return repositoryRecord;
     }
 
     @Override
-    public StandardRepositoryRecord deserializeRecord(final DataInputStream in, final int version) throws IOException {
+    public SerializedRepositoryRecord deserializeRecord(final DataInputStream in, final int version) throws IOException {
         final int action = in.read();
         if (action == -1) {
             return null;
@@ -283,8 +285,11 @@ public class WriteAheadRepositoryRecordSerde extends RepositoryRecordSerde imple
             }
 
             final FlowFileRecord flowFileRecord = ffBuilder.build();
-            final StandardRepositoryRecord record = new StandardRepositoryRecord(null, flowFileRecord);
-            record.markForDelete();
+            final SerializedRepositoryRecord record = new ReconstitutedSerializedRepositoryRecord.Builder()
+                .type(RepositoryRecordType.DELETE)
+                .flowFileRecord(flowFileRecord)
+                .build();
+
             return record;
         }
 
@@ -358,27 +363,33 @@ public class WriteAheadRepositoryRecordSerde extends RepositoryRecordSerde imple
             swapLocation = in.readUTF();
         }
 
-        final StandardRepositoryRecord record;
-        final FlowFileQueue queue = getFlowFileQueue(connectionId);
-        record = new StandardRepositoryRecord(queue, flowFile);
-        if (swapLocation != null) {
-            record.setSwapLocation(swapLocation);
-        }
-
-        if (connectionId.isEmpty()) {
-            logger.warn("{} does not have a FlowFile Queue associated with it; this record will be discarded", flowFile);
-            record.markForAbort();
-        } else if (queue == null) {
-            logger.warn("{} maps to unknown FlowFile Queue {}; this record will be discarded", flowFile, connectionId);
-            record.markForAbort();
-        }
+        final SerializedRepositoryRecord record = new ReconstitutedSerializedRepositoryRecord.Builder()
+            .queueIdentifier(connectionId)
+            .flowFileRecord(flowFile)
+            .swapLocation(swapLocation)
+            .type(getRecordType(action))
+            .build();
 
         recordsRestored++;
         return record;
     }
 
+    private RepositoryRecordType getRecordType(final int serializedUpdateType) {
+        switch (serializedUpdateType) {
+            case ACTION_CREATE:
+                return RepositoryRecordType.CREATE;
+            case ACTION_SWAPPED_IN:
+                return RepositoryRecordType.SWAP_IN;
+            case ACTION_SWAPPED_OUT:
+                return RepositoryRecordType.SWAP_OUT;
+            case ACTION_UPDATE:
+            default:
+                return RepositoryRecordType.UPDATE;
+        }
+    }
+
     @Override
-    public void serializeRecord(final RepositoryRecord record, final DataOutputStream out) throws IOException {
+    public void serializeRecord(final SerializedRepositoryRecord record, final DataOutputStream out) throws IOException {
         serializeEdit(null, record, out, true);
     }
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/schema/RepositoryRecordFieldMap.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/schema/RepositoryRecordFieldMap.java
index 5fe4889..cc1ec0d 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/schema/RepositoryRecordFieldMap.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/schema/RepositoryRecordFieldMap.java
@@ -17,23 +17,22 @@
 
 package org.apache.nifi.controller.repository.schema;
 
-import org.apache.nifi.controller.queue.FlowFileQueue;
 import org.apache.nifi.controller.repository.FlowFileRecord;
-import org.apache.nifi.controller.repository.RepositoryRecord;
+import org.apache.nifi.controller.repository.SerializedRepositoryRecord;
 import org.apache.nifi.repository.schema.Record;
 import org.apache.nifi.repository.schema.RecordSchema;
 
 public class RepositoryRecordFieldMap implements Record {
-    private final RepositoryRecord record;
+    private final SerializedRepositoryRecord record;
     private final FlowFileRecord flowFile;
     private final RecordSchema schema;
     private final RecordSchema contentClaimSchema;
 
-    public RepositoryRecordFieldMap(final RepositoryRecord record, final RecordSchema repoRecordSchema, final RecordSchema contentClaimSchema) {
+    public RepositoryRecordFieldMap(final SerializedRepositoryRecord record, final RecordSchema repoRecordSchema, final RecordSchema contentClaimSchema) {
         this.schema = repoRecordSchema;
         this.contentClaimSchema = contentClaimSchema;
         this.record = record;
-        this.flowFile = record.getCurrent();
+        this.flowFile = record.getFlowFileRecord();
     }
 
     @Override
@@ -42,7 +41,7 @@ public class RepositoryRecordFieldMap implements Record {
             case RepositoryRecordSchema.ACTION_TYPE:
                 return record.getType().name();
             case RepositoryRecordSchema.RECORD_ID:
-                return record.getCurrent().getId();
+                return record.getFlowFileRecord().getId();
             case RepositoryRecordSchema.SWAP_LOCATION:
                 return record.getSwapLocation();
             case FlowFileSchema.ATTRIBUTES:
@@ -60,12 +59,11 @@ public class RepositoryRecordFieldMap implements Record {
             case FlowFileSchema.QUEUE_DATE_INDEX:
                 return flowFile.getQueueDateIndex();
             case FlowFileSchema.CONTENT_CLAIM:
-                final ContentClaimFieldMap contentClaimFieldMap = record.getCurrentClaim() == null ? null
-                    : new ContentClaimFieldMap(record.getCurrentClaim(), record.getCurrentClaimOffset(), contentClaimSchema);
+                final ContentClaimFieldMap contentClaimFieldMap = record.getContentClaim() == null ? null
+                    : new ContentClaimFieldMap(record.getContentClaim(), record.getClaimOffset(), contentClaimSchema);
                 return contentClaimFieldMap;
             case RepositoryRecordSchema.QUEUE_IDENTIFIER:
-                final FlowFileQueue queue = record.getDestination() == null ? record.getOriginalQueue() : record.getDestination();
-                return queue == null ? null : queue.getIdentifier();
+                return record.getQueueIdentifier();
             default:
                 return null;
         }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/test/java/org/apache/nifi/controller/repository/SchemaRepositoryRecordSerdeTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/test/java/org/apache/nifi/controller/repository/SchemaRepositoryRecordSerdeTest.java
deleted file mode 100644
index af6ed9b..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/test/java/org/apache/nifi/controller/repository/SchemaRepositoryRecordSerdeTest.java
+++ /dev/null
@@ -1,292 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.nifi.controller.repository;
-
-import org.apache.nifi.controller.queue.FlowFileQueue;
-import org.apache.nifi.controller.repository.claim.StandardResourceClaimManager;
-import org.apache.nifi.controller.repository.schema.RepositoryRecordSchema;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.EOFException;
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.Map;
-
-import static org.apache.nifi.controller.repository.RepositoryRecordType.SWAP_IN;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotEquals;
-import static org.junit.Assert.assertNull;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-public class SchemaRepositoryRecordSerdeTest {
-    public static final String TEST_QUEUE_IDENTIFIER = "testQueueIdentifier";
-    private StandardResourceClaimManager resourceClaimManager;
-    private SchemaRepositoryRecordSerde schemaRepositoryRecordSerde;
-    private Map<String, FlowFileQueue> queueMap;
-    private FlowFileQueue flowFileQueue;
-    private ByteArrayOutputStream byteArrayOutputStream;
-    private DataOutputStream dataOutputStream;
-
-    @Before
-    public void setup() {
-        resourceClaimManager = new StandardResourceClaimManager();
-        schemaRepositoryRecordSerde = new SchemaRepositoryRecordSerde(resourceClaimManager);
-        queueMap = new HashMap<>();
-        schemaRepositoryRecordSerde.setQueueMap(queueMap);
-        flowFileQueue = createMockQueue(TEST_QUEUE_IDENTIFIER);
-        byteArrayOutputStream = new ByteArrayOutputStream();
-        dataOutputStream = new DataOutputStream(byteArrayOutputStream);
-    }
-
-    @After
-    public void teardown() {
-        resourceClaimManager.purge();
-    }
-
-    @Test
-    public void testV1CreateCantHandleLongAttributeName() throws IOException {
-        RepositoryRecordSchema.REPOSITORY_RECORD_SCHEMA_V1.writeTo(dataOutputStream);
-        Map<String, String> attributes = new HashMap<>();
-        StringBuilder stringBuilder = new StringBuilder();
-        for (int i = 0; i < 65536; i++) {
-            stringBuilder.append('a');
-        }
-        attributes.put(stringBuilder.toString(), "testValue");
-        schemaRepositoryRecordSerde.serializeRecord(createCreateFlowFileRecord(attributes), dataOutputStream,
-                RepositoryRecordSchema.CREATE_OR_UPDATE_SCHEMA_V1, RepositoryRecordSchema.REPOSITORY_RECORD_SCHEMA_V1);
-
-        DataInputStream dataInputStream = createDataInputStream();
-        schemaRepositoryRecordSerde.readHeader(dataInputStream);
-        RepositoryRecord repositoryRecord = schemaRepositoryRecordSerde.deserializeRecord(dataInputStream, 2);
-        assertNotEquals(attributes, repositoryRecord.getCurrent().getAttributes());
-    }
-
-    @Test
-    public void testV1CreateCantHandleLongAttributeValue() throws IOException {
-        RepositoryRecordSchema.REPOSITORY_RECORD_SCHEMA_V1.writeTo(dataOutputStream);
-        Map<String, String> attributes = new HashMap<>();
-        StringBuilder stringBuilder = new StringBuilder();
-        for (int i = 0; i < 65536; i++) {
-            stringBuilder.append('a');
-        }
-        attributes.put("testName", stringBuilder.toString());
-        schemaRepositoryRecordSerde.serializeRecord(createCreateFlowFileRecord(attributes), dataOutputStream,
-                RepositoryRecordSchema.CREATE_OR_UPDATE_SCHEMA_V1, RepositoryRecordSchema.REPOSITORY_RECORD_SCHEMA_V1);
-
-        DataInputStream dataInputStream = createDataInputStream();
-        schemaRepositoryRecordSerde.readHeader(dataInputStream);
-        RepositoryRecord repositoryRecord = schemaRepositoryRecordSerde.deserializeRecord(dataInputStream, 2);
-        assertNotEquals(attributes, repositoryRecord.getCurrent().getAttributes());
-    }
-
-    @Test
-    public void testV2CreateCanHandleLongAttributeName() throws IOException {
-        schemaRepositoryRecordSerde.writeHeader(dataOutputStream);
-        Map<String, String> attributes = new HashMap<>();
-        StringBuilder stringBuilder = new StringBuilder();
-        for (int i = 0; i < 65536; i++) {
-            stringBuilder.append('a');
-        }
-        attributes.put(stringBuilder.toString(), "testValue");
-        schemaRepositoryRecordSerde.serializeRecord(createCreateFlowFileRecord(attributes), dataOutputStream);
-
-        DataInputStream dataInputStream = createDataInputStream();
-        schemaRepositoryRecordSerde.readHeader(dataInputStream);
-        RepositoryRecord repositoryRecord = schemaRepositoryRecordSerde.deserializeRecord(dataInputStream, 2);
-        assertEquals(attributes, repositoryRecord.getCurrent().getAttributes());
-    }
-
-    @Test
-    public void testV2CreateCanHandleLongAttributeValue() throws IOException {
-        schemaRepositoryRecordSerde.writeHeader(dataOutputStream);
-        Map<String, String> attributes = new HashMap<>();
-        StringBuilder stringBuilder = new StringBuilder();
-        for (int i = 0; i < 65536; i++) {
-            stringBuilder.append('a');
-        }
-        attributes.put("testName", stringBuilder.toString());
-        schemaRepositoryRecordSerde.serializeRecord(createCreateFlowFileRecord(attributes), dataOutputStream);
-
-        DataInputStream dataInputStream = createDataInputStream();
-        schemaRepositoryRecordSerde.readHeader(dataInputStream);
-        RepositoryRecord repositoryRecord = schemaRepositoryRecordSerde.deserializeRecord(dataInputStream, 2);
-        assertEquals(attributes, repositoryRecord.getCurrent().getAttributes());
-    }
-
-    @Test
-    public void testRoundTripCreateV1ToV2() throws IOException {
-        RepositoryRecordSchema.REPOSITORY_RECORD_SCHEMA_V1.writeTo(dataOutputStream);
-        Map<String, String> attributes = new HashMap<>();
-        attributes.put("testName", "testValue");
-        schemaRepositoryRecordSerde.serializeRecord(createCreateFlowFileRecord(attributes), dataOutputStream,
-                RepositoryRecordSchema.CREATE_OR_UPDATE_SCHEMA_V1, RepositoryRecordSchema.REPOSITORY_RECORD_SCHEMA_V1);
-
-        DataInputStream dataInputStream = createDataInputStream();
-        schemaRepositoryRecordSerde.readHeader(dataInputStream);
-        RepositoryRecord repositoryRecord = schemaRepositoryRecordSerde.deserializeRecord(dataInputStream, 2);
-        assertEquals(attributes, repositoryRecord.getCurrent().getAttributes());
-    }
-
-    @Test
-    public void testV1SwapInCantHandleLongAttributeName() throws IOException {
-        RepositoryRecordSchema.REPOSITORY_RECORD_SCHEMA_V1.writeTo(dataOutputStream);
-        Map<String, String> attributes = new HashMap<>();
-        StringBuilder stringBuilder = new StringBuilder();
-        for (int i = 0; i < 65536; i++) {
-            stringBuilder.append('a');
-        }
-        attributes.put(stringBuilder.toString(), "testValue");
-        StandardRepositoryRecord record = createCreateFlowFileRecord(attributes);
-        record.setSwapLocation("fake");
-        assertEquals(SWAP_IN, record.getType());
-        schemaRepositoryRecordSerde.serializeRecord(record, dataOutputStream, RepositoryRecordSchema.SWAP_IN_SCHEMA_V1, RepositoryRecordSchema.REPOSITORY_RECORD_SCHEMA_V1);
-
-        DataInputStream dataInputStream = createDataInputStream();
-        schemaRepositoryRecordSerde.readHeader(dataInputStream);
-        RepositoryRecord repositoryRecord = schemaRepositoryRecordSerde.deserializeRecord(dataInputStream, 2);
-        assertNotEquals(attributes, repositoryRecord.getCurrent().getAttributes());
-    }
-
-    @Test
-    public void testV1SwapInCantHandleLongAttributeValue() throws IOException {
-        RepositoryRecordSchema.REPOSITORY_RECORD_SCHEMA_V1.writeTo(dataOutputStream);
-        Map<String, String> attributes = new HashMap<>();
-        StringBuilder stringBuilder = new StringBuilder();
-        for (int i = 0; i < 65536; i++) {
-            stringBuilder.append('a');
-        }
-        attributes.put("testName", stringBuilder.toString());
-        StandardRepositoryRecord record = createCreateFlowFileRecord(attributes);
-        record.setSwapLocation("fake");
-        assertEquals(SWAP_IN, record.getType());
-        schemaRepositoryRecordSerde.serializeRecord(record, dataOutputStream, RepositoryRecordSchema.SWAP_IN_SCHEMA_V1, RepositoryRecordSchema.REPOSITORY_RECORD_SCHEMA_V1);
-
-        DataInputStream dataInputStream = createDataInputStream();
-        schemaRepositoryRecordSerde.readHeader(dataInputStream);
-        RepositoryRecord repositoryRecord = schemaRepositoryRecordSerde.deserializeRecord(dataInputStream, 2);
-        assertNotEquals(attributes, repositoryRecord.getCurrent().getAttributes());
-    }
-
-    @Test
-    public void testV2SwapInCanHandleLongAttributeName() throws IOException {
-        schemaRepositoryRecordSerde.writeHeader(dataOutputStream);
-        Map<String, String> attributes = new HashMap<>();
-        StringBuilder stringBuilder = new StringBuilder();
-        for (int i = 0; i < 65536; i++) {
-            stringBuilder.append('a');
-        }
-        attributes.put(stringBuilder.toString(), "testValue");
-        StandardRepositoryRecord record = createCreateFlowFileRecord(attributes);
-        record.setSwapLocation("fake");
-        assertEquals(SWAP_IN, record.getType());
-        schemaRepositoryRecordSerde.serializeRecord(record, dataOutputStream);
-
-        DataInputStream dataInputStream = createDataInputStream();
-        schemaRepositoryRecordSerde.readHeader(dataInputStream);
-        RepositoryRecord repositoryRecord = schemaRepositoryRecordSerde.deserializeRecord(dataInputStream, 2);
-        assertEquals(attributes, repositoryRecord.getCurrent().getAttributes());
-    }
-
-    @Test
-    public void testV2SwapInCanHandleLongAttributeValue() throws IOException {
-        schemaRepositoryRecordSerde.writeHeader(dataOutputStream);
-        Map<String, String> attributes = new HashMap<>();
-        StringBuilder stringBuilder = new StringBuilder();
-        for (int i = 0; i < 65536; i++) {
-            stringBuilder.append('a');
-        }
-        attributes.put("testName", stringBuilder.toString());
-        StandardRepositoryRecord record = createCreateFlowFileRecord(attributes);
-        record.setSwapLocation("fake");
-        assertEquals(SWAP_IN, record.getType());
-        schemaRepositoryRecordSerde.serializeRecord(record, dataOutputStream);
-
-        DataInputStream dataInputStream = createDataInputStream();
-        schemaRepositoryRecordSerde.readHeader(dataInputStream);
-        RepositoryRecord repositoryRecord = schemaRepositoryRecordSerde.deserializeRecord(dataInputStream, 2);
-        assertEquals(attributes, repositoryRecord.getCurrent().getAttributes());
-    }
-
-    @Test
-    public void testRoundTripSwapInV1ToV2() throws IOException {
-        RepositoryRecordSchema.REPOSITORY_RECORD_SCHEMA_V1.writeTo(dataOutputStream);
-        Map<String, String> attributes = new HashMap<>();
-        attributes.put("testName", "testValue");
-        StandardRepositoryRecord record = createCreateFlowFileRecord(attributes);
-        record.setSwapLocation("fake");
-        assertEquals(SWAP_IN, record.getType());
-        schemaRepositoryRecordSerde.serializeRecord(record, dataOutputStream, RepositoryRecordSchema.SWAP_IN_SCHEMA_V1, RepositoryRecordSchema.REPOSITORY_RECORD_SCHEMA_V1);
-
-        DataInputStream dataInputStream = createDataInputStream();
-        schemaRepositoryRecordSerde.readHeader(dataInputStream);
-        RepositoryRecord repositoryRecord = schemaRepositoryRecordSerde.deserializeRecord(dataInputStream, 2);
-        assertEquals(attributes, repositoryRecord.getCurrent().getAttributes());
-        assertEquals(SWAP_IN, repositoryRecord.getType());
-    }
-
-    @Test
-    public void testEOFExceptionOnDeserializeEdit() throws IOException {
-        RepositoryRecordSchema.REPOSITORY_RECORD_SCHEMA_V1.writeTo(dataOutputStream);
-
-        DataInputStream dataInputStream = createDataInputStream();
-        schemaRepositoryRecordSerde.readHeader(dataInputStream);
-
-        // calling deserializeRecord on an empty stream should return a null record.
-        RepositoryRecord repositoryRecord = schemaRepositoryRecordSerde.deserializeRecord(dataInputStream, 2);
-        assertNull(repositoryRecord);
-
-        dataInputStream = createDataInputStream();
-        schemaRepositoryRecordSerde.readHeader(dataInputStream);
-
-        // calling deserializeEdit on an empty stream should throw EOFException
-        try {
-            schemaRepositoryRecordSerde.deserializeEdit(dataInputStream, new HashMap<>(), 2);
-            Assert.fail("Expected EOFException");
-        } catch (final EOFException eof) {
-            // expected
-        }
-    }
-
-    private DataInputStream createDataInputStream() throws IOException {
-        dataOutputStream.flush();
-        return new DataInputStream(new ByteArrayInputStream(byteArrayOutputStream.toByteArray()));
-    }
-
-    private StandardRepositoryRecord createCreateFlowFileRecord(Map<String, String> attributes) {
-        StandardRepositoryRecord standardRepositoryRecord = new StandardRepositoryRecord(flowFileQueue);
-        StandardFlowFileRecord.Builder flowFileRecordBuilder = new StandardFlowFileRecord.Builder();
-        flowFileRecordBuilder.addAttributes(attributes);
-        standardRepositoryRecord.setWorking(flowFileRecordBuilder.build());
-        return standardRepositoryRecord;
-    }
-
-    private FlowFileQueue createMockQueue(String identifier) {
-        FlowFileQueue flowFileQueue = mock(FlowFileQueue.class);
-        when(flowFileQueue.getIdentifier()).thenReturn(identifier);
-        queueMap.put(identifier, flowFileQueue);
-        return flowFileQueue;
-    }
-}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-authorization/src/main/java/org/apache/nifi/authorization/CompositeConfigurableUserGroupProvider.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-authorization/src/main/java/org/apache/nifi/authorization/CompositeConfigurableUserGroupProvider.java
index 91c4186..b9ecf9d 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-authorization/src/main/java/org/apache/nifi/authorization/CompositeConfigurableUserGroupProvider.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-authorization/src/main/java/org/apache/nifi/authorization/CompositeConfigurableUserGroupProvider.java
@@ -92,6 +92,11 @@ public class CompositeConfigurableUserGroupProvider extends CompositeUserGroupPr
     }
 
     @Override
+    public void forciblyInheritFingerprint(final String fingerprint) throws AuthorizationAccessException {
+        configurableUserGroupProvider.forciblyInheritFingerprint(fingerprint);
+    }
+
+    @Override
     public void checkInheritability(String proposedFingerprint) throws AuthorizationAccessException, UninheritableAuthorizationsException {
         configurableUserGroupProvider.checkInheritability(proposedFingerprint);
     }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-authorization/src/main/java/org/apache/nifi/authorization/StandardManagedAuthorizer.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-authorization/src/main/java/org/apache/nifi/authorization/StandardManagedAuthorizer.java
index 4dfe040..7c1e9c8 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-authorization/src/main/java/org/apache/nifi/authorization/StandardManagedAuthorizer.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-authorization/src/main/java/org/apache/nifi/authorization/StandardManagedAuthorizer.java
@@ -180,6 +180,19 @@ public class StandardManagedAuthorizer implements ManagedAuthorizer {
     }
 
     @Override
+    public void forciblyInheritFingerprint(final String fingerprint) throws AuthorizationAccessException {
+        final FingerprintHolder fingerprintHolder = parseFingerprint(fingerprint);
+
+        if (accessPolicyProvider instanceof ConfigurableAccessPolicyProvider) {
+            ((ConfigurableAccessPolicyProvider) accessPolicyProvider).forciblyInheritFingerprint(fingerprintHolder.getPolicyFingerprint());
+        }
+
+        if (userGroupProvider instanceof ConfigurableUserGroupProvider) {
+            ((ConfigurableUserGroupProvider) userGroupProvider).forciblyInheritFingerprint(fingerprintHolder.getUserGroupFingerprint());
+        }
+    }
+
+    @Override
     public void checkInheritability(String proposedFingerprint) throws AuthorizationAccessException, UninheritableAuthorizationsException {
         final FingerprintHolder fingerprintHolder = parseFingerprint(proposedFingerprint);
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-authorization/src/test/java/org/apache/nifi/authorization/SimpleConfigurableUserGroupProvider.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-authorization/src/test/java/org/apache/nifi/authorization/SimpleConfigurableUserGroupProvider.java
index 791d968..2e84c103 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-authorization/src/test/java/org/apache/nifi/authorization/SimpleConfigurableUserGroupProvider.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-authorization/src/test/java/org/apache/nifi/authorization/SimpleConfigurableUserGroupProvider.java
@@ -82,6 +82,11 @@ public class SimpleConfigurableUserGroupProvider extends SimpleUserGroupProvider
     }
 
     @Override
+    public void forciblyInheritFingerprint(final String fingerprint) throws AuthorizationAccessException {
+
+    }
+
+    @Override
     public void checkInheritability(String proposedFingerprint) throws AuthorizationAccessException, UninheritableAuthorizationsException {
 
     }
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 7822db7..cadaad0 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,28 +16,44 @@
  */
 package org.apache.nifi.cluster.protocol;
 
+import org.apache.nifi.cluster.protocol.jaxb.message.DataFlowAdapter;
+import org.apache.nifi.controller.serialization.FlowSerializationException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.w3c.dom.Document;
+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.DocumentBuilderFactory;
+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;
 
-import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
-
-import org.apache.nifi.cluster.protocol.jaxb.message.DataFlowAdapter;
-
 /**
  * Represents a dataflow, which includes the raw bytes of the flow.xml and
  * whether processors should be started automatically at application startup.
  */
 @XmlJavaTypeAdapter(DataFlowAdapter.class)
 public class StandardDataFlow implements Serializable, DataFlow {
+    private static final URL FLOW_XSD_RESOURCE = StandardDataFlow.class.getClassLoader().getResource("/FlowConfiguration.xsd");
+    private static final Logger logger = LoggerFactory.getLogger(StandardDataFlow.class);
+
     private static final long serialVersionUID = 1L;
 
     private final byte[] flow;
     private final byte[] snippetBytes;
     private final byte[] authorizerFingerprint;
     private final Set<String> missingComponentIds;
+    private Document flowDocument;
 
     /**
      * Constructs an instance.
@@ -56,16 +72,14 @@ public class StandardDataFlow implements Serializable, DataFlow {
         this.flow = flow;
         this.snippetBytes = snippetBytes;
         this.authorizerFingerprint = authorizerFingerprint;
-        this.missingComponentIds = Collections.unmodifiableSet(missingComponentIds == null
-                ? new HashSet<>() : new HashSet<>(missingComponentIds));
+        this.missingComponentIds = Collections.unmodifiableSet(missingComponentIds == null ? new HashSet<>() : new HashSet<>(missingComponentIds));
     }
 
     public StandardDataFlow(final DataFlow toCopy) {
         this.flow = copy(toCopy.getFlow());
         this.snippetBytes = copy(toCopy.getSnippets());
         this.authorizerFingerprint = copy(toCopy.getAuthorizerFingerprint());
-        this.missingComponentIds = Collections.unmodifiableSet(toCopy.getMissingComponents() == null
-                ? new HashSet<>() : new HashSet<>(toCopy.getMissingComponents()));
+        this.missingComponentIds = Collections.unmodifiableSet(toCopy.getMissingComponents() == null ? new HashSet<>() : new HashSet<>(toCopy.getMissingComponents()));
     }
 
     private static byte[] copy(final byte[] bytes) {
@@ -77,6 +91,14 @@ public class StandardDataFlow implements Serializable, DataFlow {
         return flow;
     }
 
+    @Override
+    public synchronized Document getFlowDocument() {
+        if (flowDocument == null) {
+            flowDocument = parseFlowBytes(flow);
+        }
+
+        return flowDocument;
+    }
 
     @Override
     public byte[] getSnippets() {
@@ -93,4 +115,29 @@ public class StandardDataFlow implements Serializable, DataFlow {
         return missingComponentIds;
     }
 
+    private static Document parseFlowBytes(final byte[] flow) throws FlowSerializationException {
+        if (flow == null || flow.length == 0) {
+            return null;
+        }
+
+        // create document by parsing proposed flow bytes
+        try {
+            // create validating document builder
+            final DocumentBuilderFactory docFactory = DocumentBuilderFactory.newInstance();
+            docFactory.setNamespaceAware(true);
+
+            final DocumentBuilder docBuilder = docFactory.newDocumentBuilder();
+            docBuilder.setErrorHandler(new DefaultHandler() {
+                @Override
+                public void error(final SAXParseException e) {
+                    logger.warn("Schema validation error parsing Flow Configuration at line {}, col {}: {}", e.getLineNumber(), e.getColumnNumber(), e.getMessage());
+                }
+            });
+
+            // parse flow
+            return docBuilder.parse(new ByteArrayInputStream(flow));
+        } catch (final SAXException | ParserConfigurationException | IOException ex) {
+            throw new FlowSerializationException(ex);
+        }
+    }
 }
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 0317584..734cc3c 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,8 @@
  */
 package org.apache.nifi.cluster.protocol;
 
+import org.w3c.dom.Document;
+
 import java.util.Set;
 
 public interface DataFlow {
@@ -23,22 +25,24 @@ public interface DataFlow {
     /**
      * @return the raw byte array of the flow
      */
-    public byte[] getFlow();
+    byte[] getFlow();
+
+    Document getFlowDocument();
 
     /**
      * @return the raw byte array of the snippets
      */
-    public byte[] getSnippets();
+    byte[] getSnippets();
 
     /**
      * @return the raw byte array of the Authorizer's fingerprint,
      *              null when not using a ManagedAuthorizer
      */
-    public byte[] getAuthorizerFingerprint();
+    byte[] getAuthorizerFingerprint();
 
     /**
      * @return the component ids of components that were created as a missing ghost component
      */
-    public Set<String> getMissingComponents();
+    Set<String> getMissingComponents();
 
 }
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 d072698..ba24705 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
@@ -326,4 +326,18 @@ public interface FlowManager {
     ParameterContext createParameterContext(String id, String name, Map<String, Parameter> parameters);
 
     ParameterContextManager getParameterContextManager();
+
+    /**
+     * Purges all components from the flow, including:
+     *
+     * Process Groups (and all components within it)
+     * Controller Services
+     * Templates
+     * Reporting Tasks
+     * Parameter Contexts
+     * Flow Registries
+     *
+     * @throws IllegalStateException if any of the components is not in a state that it can be deleted.
+     */
+    void purge();
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/FlowSerializationException.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/serialization/FlowSerializationException.java
similarity index 100%
rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/FlowSerializationException.java
rename to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/serialization/FlowSerializationException.java
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 f4ec742..a386b49 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
@@ -848,6 +848,20 @@ public interface ProcessGroup extends ComponentAuthorizable, Positionable, Versi
      */
     void verifyCanDelete(boolean ignorePortConnections);
 
+
+    /**
+     * Ensures that the ProcessGroup is eligible to be deleted.
+     *
+     * @param ignorePortConnections if true, the Connections that are currently connected to Ports
+     * will be ignored. Otherwise, the ProcessGroup is not eligible for deletion if its input ports
+     * or output ports have any connections
+     * @param ignoreTemplates if true, the Templates that are currently part of hte Process Group will be ignored.
+     * Otherwise, the ProcessGroup is not eligible for deletion if it has any templates
+     *
+     * @throws IllegalStateException if the ProcessGroup is not eligible for deletion
+     */
+    void verifyCanDelete(boolean ignorePortConnections, boolean ignoreTemplates);
+
     void verifyCanStart(Connectable connectable);
 
     void verifyCanStart();
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/services/FlowService.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/services/FlowService.java
index 4f5d284..8d69550 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/services/FlowService.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/services/FlowService.java
@@ -16,14 +16,14 @@
  */
 package org.apache.nifi.services;
 
+import org.apache.nifi.cluster.protocol.DataFlow;
+import org.apache.nifi.lifecycle.LifeCycle;
+
+import java.io.File;
 import java.io.IOException;
-import java.io.InputStream;
 import java.io.OutputStream;
 import java.util.concurrent.TimeUnit;
 
-import org.apache.nifi.cluster.protocol.DataFlow;
-import org.apache.nifi.lifecycle.LifeCycle;
-
 /**
  * Defines the API level services available for carrying out file-based dataflow operations.
  *
@@ -39,23 +39,6 @@ public interface FlowService extends LifeCycle {
     void saveFlowChanges() throws IOException;
 
     /**
-     * Immediately persists the state of the flow controller to the given output stream in a blocking call.
-     *
-     * @param outStream the stream to which the FlowController is to be persisted
-     * @throws NullPointerException if the given flow is null.
-     * @throws IOException if any problem occurs creating/modifying file
-     */
-    void saveFlowChanges(OutputStream outStream) throws IOException;
-
-    /**
-     * Saves the given stream to the flow.xml file on disk. This method does not change the state of the flow controller.
-     *
-     * @param is an input stream
-     * @throws IOException if unable to save the flow
-     */
-    void overwriteFlow(InputStream is) throws IOException;
-
-    /**
      * Asynchronously saves the flow controller. The flow controller will be copied and immediately returned. If another call to save is made within that time the latest called state of the flow
      * controller will be used. In database terms this technique is referred to as 'write-delay'.
      *
@@ -103,6 +86,13 @@ public interface FlowService extends LifeCycle {
     void copyCurrentFlow(OutputStream os) throws IOException;
 
     /**
+     * Copies the contents of the current flow.xml.gz to the given file, overwriting the file if it exists
+     * @param file the file to write the current flow to
+     * @throws IOException if unable to read the current flow or unable to write to the given file
+     */
+    void copyCurrentFlow(File file) throws IOException;
+
+    /**
      * Creates a DataFlow object by first looking for a flow on from disk, and falling back to the controller's flow otherwise.
      *
      * @return the created DataFlow object
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 1ff685d..72c3416 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
@@ -206,6 +206,7 @@ import org.apache.nifi.reporting.Severity;
 import org.apache.nifi.reporting.StandardEventAccess;
 import org.apache.nifi.reporting.UserAwareEventAccess;
 import org.apache.nifi.scheduling.SchedulingStrategy;
+import org.apache.nifi.services.FlowService;
 import org.apache.nifi.stream.io.LimitingInputStream;
 import org.apache.nifi.stream.io.StreamUtils;
 import org.apache.nifi.util.ComponentIdGenerator;
@@ -220,6 +221,39 @@ import org.apache.zookeeper.server.quorum.QuorumPeerConfig.ConfigException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import javax.management.NotificationEmitter;
+import javax.net.ssl.SSLContext;
+import java.io.ByteArrayInputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.lang.management.GarbageCollectorMXBean;
+import java.lang.management.ManagementFactory;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.stream.Collectors;
+
+import static java.util.Objects.requireNonNull;
+
 public class FlowController implements ReportingTaskProvider, Authorizable, NodeTypeProvider {
 
     // default repository implementations
@@ -821,6 +855,18 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
         };
     }
 
+    public void purge() {
+        getFlowManager().purge();
+
+        writeLock.lock();
+        try {
+            startConnectablesAfterInitialization.clear();
+            startRemoteGroupPortsAfterInitialization.clear();
+        } finally {
+            writeLock.unlock("purge");
+        }
+    }
+
     public void initializeFlow() throws IOException {
         initializeFlow(new StandardQueueProvider(getFlowManager()));
     }
@@ -1352,28 +1398,46 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
      * Synchronizes this controller with the proposed flow.
      * <p>
      * For more details, see
-     * {@link FlowSynchronizer#sync(FlowController, DataFlow, StringEncryptor)}.
+     * {@link FlowSynchronizer#sync(FlowController, DataFlow, StringEncryptor, FlowService)}.
      *
      * @param synchronizer synchronizer
-     * @param dataFlow     the flow to load the controller with. If the flow is null
-     *                     or zero length, then the controller must not have a flow or else an
-     *                     UninheritableFlowException will be thrown.
-     * @throws FlowSerializationException   if proposed flow is not a valid flow
-     *                                      configuration file
-     * @throws UninheritableFlowException   if the proposed flow cannot be loaded
-     *                                      by the controller because in doing so would risk orphaning flow files
+     * @param dataFlow the flow to load the controller with. If the flow is null
+     * or zero length, then the controller must not have a flow or else an
+     * UninheritableFlowException will be thrown.
+     * @param flowService the flow service
+     *
+     * @throws FlowSerializationException if proposed flow is not a valid flow
+     * configuration file
+     * @throws UninheritableFlowException if the proposed flow cannot be loaded
+     * by the controller because in doing so would risk orphaning flow files
      * @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 does not exist in the controller
      */
-    public void synchronize(final FlowSynchronizer synchronizer, final DataFlow dataFlow)
+    public void synchronize(final FlowSynchronizer synchronizer, final DataFlow dataFlow, final FlowService flowService)
             throws FlowSerializationException, FlowSynchronizationException, UninheritableFlowException, MissingBundleException {
         writeLock.lock();
         try {
             LOG.debug("Synchronizing controller with proposed flow");
-            synchronizer.sync(this, dataFlow, encryptor);
+
+            try {
+                synchronizer.sync(this, dataFlow, encryptor, flowService);
+            } catch (final UninheritableFlowException ufe) {
+                final NodeIdentifier localNodeId = getNodeId();
+                if (localNodeId != null) {
+                    try {
+                        clusterCoordinator.requestNodeDisconnect(localNodeId, DisconnectionCode.MISMATCHED_FLOWS, ufe.getMessage());
+                    } catch (final Exception e) {
+                        LOG.error("Failed to synchronize Controller with proposed flow and also failed to notify cluster that the flows do not match. Node's state may remain CONNECTING instead of " +
+                            "transitioning to DISCONNECTED.", e);
+                    }
+                }
+
+                throw ufe;
+            }
+
             flowSynchronized.set(true);
             LOG.info("Successfully synchronized controller with proposed flow");
         } finally {
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 ef50439..aa86672 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
@@ -66,6 +66,7 @@ import org.apache.nifi.persistence.TemplateDeserializer;
 import org.apache.nifi.reporting.Bulletin;
 import org.apache.nifi.reporting.EventAccess;
 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;
@@ -78,6 +79,7 @@ import java.io.BufferedInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.File;
 import java.io.FileInputStream;
+import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
@@ -104,7 +106,6 @@ import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 import java.util.stream.Collectors;
 import java.util.zip.GZIPInputStream;
-import java.util.zip.GZIPOutputStream;
 
 public class StandardFlowService implements FlowService, ProtocolHandler {
 
@@ -152,7 +153,7 @@ public class StandardFlowService implements FlowService, ProtocolHandler {
 
     private final NiFiProperties nifiProperties;
 
-    private static final String CONNECTION_EXCEPTION_MSG_PREFIX = "Failed to connect node to cluster because ";
+    private static final String CONNECTION_EXCEPTION_MSG_PREFIX = "Failed to connect node to cluster";
     private static final Logger logger = LoggerFactory.getLogger(StandardFlowService.class);
 
     public static StandardFlowService createStandaloneInstance(
@@ -248,27 +249,6 @@ public class StandardFlowService implements FlowService, ProtocolHandler {
     }
 
     @Override
-    public void saveFlowChanges(final OutputStream outStream) throws IOException {
-        writeLock.lock();
-        try {
-            dao.save(controller, outStream);
-        } finally {
-            writeLock.unlock();
-        }
-    }
-
-    @Override
-    public void overwriteFlow(final InputStream is) throws IOException {
-        writeLock.lock();
-        try (final OutputStream output = Files.newOutputStream(flowXml, StandardOpenOption.WRITE, StandardOpenOption.CREATE);
-                final OutputStream gzipOut = new GZIPOutputStream(output)) {
-            FileUtils.copy(is, gzipOut);
-        } finally {
-            writeLock.unlock();
-        }
-    }
-
-    @Override
     public void saveFlowChanges(final TimeUnit delayUnit, final long delay) {
         final boolean archiveEnabled = nifiProperties.isFlowConfigurationArchiveEnabled();
         saveFlowChanges(delayUnit, delay, archiveEnabled);
@@ -809,7 +789,7 @@ public class StandardFlowService implements FlowService, ProtocolHandler {
 
         // load the flow
         logger.debug("Loading proposed flow into FlowController");
-        dao.load(controller, actualProposedFlow);
+        dao.load(controller, actualProposedFlow, this);
 
         final ProcessGroup rootGroup = controller.getFlowManager().getRootGroup();
         if (rootGroup.isEmpty() && !allowEmptyFlow) {
@@ -997,13 +977,14 @@ public class StandardFlowService implements FlowService, ProtocolHandler {
                 logger.trace("ResponseFlow = " + new String(dataFlow.getFlow(), StandardCharsets.UTF_8));
             }
 
+            logger.info("Setting Flow Controller's Node ID: " + nodeId);
+            nodeId = response.getNodeIdentifier();
+            controller.setNodeId(nodeId);
+
             // load new controller state
             loadFromBytes(dataFlow, true);
 
             // set node ID on controller before we start heartbeating because heartbeat needs node ID
-            nodeId = response.getNodeIdentifier();
-            logger.info("Setting Flow Controller's Node ID: " + nodeId);
-            controller.setNodeId(nodeId);
             clusterCoordinator.setLocalNodeIdentifier(nodeId);
             clusterCoordinator.setConnected(true);
             revisionManager.reset(response.getComponentRevisions().stream().map(ComponentRevision::toRevision).collect(Collectors.toList()));
@@ -1023,13 +1004,13 @@ public class StandardFlowService implements FlowService, ProtocolHandler {
 
             controller.startHeartbeating();
         } catch (final UninheritableFlowException ufe) {
-            throw new UninheritableFlowException(CONNECTION_EXCEPTION_MSG_PREFIX + "local flow is different than cluster flow.", ufe);
+            throw new UninheritableFlowException(CONNECTION_EXCEPTION_MSG_PREFIX, ufe);
         } catch (final MissingBundleException mbe) {
-            throw new MissingBundleException(CONNECTION_EXCEPTION_MSG_PREFIX + "cluster flow contains bundles that do not exist on the current node", mbe);
+            throw new MissingBundleException(CONNECTION_EXCEPTION_MSG_PREFIX + " because cluster flow contains bundles that do not exist on the current node", mbe);
         } catch (final FlowSerializationException fse) {
-            throw new ConnectionException(CONNECTION_EXCEPTION_MSG_PREFIX + "local or cluster flow is malformed.", fse);
+            throw new ConnectionException(CONNECTION_EXCEPTION_MSG_PREFIX + " because local or cluster flow is malformed.", fse);
         } catch (final FlowSynchronizationException fse) {
-            throw new FlowSynchronizationException(CONNECTION_EXCEPTION_MSG_PREFIX + "local flow controller partially updated. "
+            throw new FlowSynchronizationException(CONNECTION_EXCEPTION_MSG_PREFIX + " because local flow controller partially updated. "
                     + "Administrator should disconnect node and review flow for corruption.", fse);
         } catch (final Exception ex) {
             throw new ConnectionException("Failed to connect node to cluster due to: " + ex, ex);
@@ -1064,6 +1045,14 @@ public class StandardFlowService implements FlowService, ProtocolHandler {
         }
     }
 
+    @Override
+    public void copyCurrentFlow(final File file) throws IOException {
+        try (final OutputStream fos = new FileOutputStream(file);
+             final OutputStream gzipOut = new GZIPOutputStream(fos, 1)) {
+            copyCurrentFlow(gzipOut);
+        }
+    }
+
     public void loadSnippets(final byte[] bytes) {
         if (bytes.length == 0) {
             return;
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/StandardFlowSynchronizer.java
index c14c2df..c9460f8 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/StandardFlowSynchronizer.java
@@ -21,7 +21,6 @@ import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.authorization.Authorizer;
 import org.apache.nifi.authorization.AuthorizerCapabilityDetection;
 import org.apache.nifi.authorization.ManagedAuthorizer;
-import org.apache.nifi.authorization.exception.UninheritableAuthorizationsException;
 import org.apache.nifi.bundle.BundleCoordinate;
 import org.apache.nifi.cluster.protocol.DataFlow;
 import org.apache.nifi.cluster.protocol.StandardDataFlow;
@@ -34,6 +33,13 @@ import org.apache.nifi.connectable.Port;
 import org.apache.nifi.connectable.Position;
 import org.apache.nifi.connectable.Size;
 import org.apache.nifi.controller.flow.FlowManager;
+import org.apache.nifi.controller.inheritance.AuthorizerCheck;
+import org.apache.nifi.controller.inheritance.BundleCompatibilityCheck;
+import org.apache.nifi.controller.inheritance.ConnectionMissingCheck;
+import org.apache.nifi.controller.inheritance.FlowFingerprintCheck;
+import org.apache.nifi.controller.inheritance.FlowInheritability;
+import org.apache.nifi.controller.inheritance.FlowInheritabilityCheck;
+import org.apache.nifi.controller.inheritance.MissingComponentsCheck;
 import org.apache.nifi.controller.label.Label;
 import org.apache.nifi.controller.queue.LoadBalanceCompression;
 import org.apache.nifi.controller.queue.LoadBalanceStrategy;
@@ -50,8 +56,6 @@ import org.apache.nifi.controller.service.ControllerServiceProvider;
 import org.apache.nifi.controller.service.ControllerServiceState;
 import org.apache.nifi.encrypt.StringEncryptor;
 import org.apache.nifi.events.BulletinFactory;
-import org.apache.nifi.fingerprint.FingerprintException;
-import org.apache.nifi.fingerprint.FingerprintFactory;
 import org.apache.nifi.flowfile.FlowFilePrioritizer;
 import org.apache.nifi.groups.ProcessGroup;
 import org.apache.nifi.groups.RemoteProcessGroup;
@@ -73,9 +77,9 @@ import org.apache.nifi.remote.protocol.SiteToSiteTransportProtocol;
 import org.apache.nifi.reporting.Severity;
 import org.apache.nifi.scheduling.ExecutionNode;
 import org.apache.nifi.scheduling.SchedulingStrategy;
+import org.apache.nifi.services.FlowService;
 import org.apache.nifi.util.BundleUtils;
 import org.apache.nifi.util.DomUtils;
-import org.apache.nifi.util.LoggingXmlParserErrorHandler;
 import org.apache.nifi.util.NiFiProperties;
 import org.apache.nifi.util.file.FileUtils;
 import org.apache.nifi.web.api.dto.BundleDTO;
@@ -104,26 +108,20 @@ import org.w3c.dom.Document;
 import org.w3c.dom.Element;
 import org.w3c.dom.Node;
 import org.w3c.dom.NodeList;
-import org.xml.sax.SAXException;
-
-import javax.xml.XMLConstants;
-import javax.xml.parsers.DocumentBuilder;
-import javax.xml.parsers.DocumentBuilderFactory;
-import javax.xml.parsers.ParserConfigurationException;
-import javax.xml.validation.Schema;
-import javax.xml.validation.SchemaFactory;
-import java.io.ByteArrayInputStream;
+
 import java.io.ByteArrayOutputStream;
+import java.io.File;
 import java.io.IOException;
 import java.io.InputStream;
-import java.net.URL;
 import java.nio.charset.StandardCharsets;
 import java.nio.file.Files;
 import java.nio.file.Path;
 import java.nio.file.StandardOpenOption;
+import java.text.DateFormat;
+import java.text.SimpleDateFormat;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Collections;
+import java.util.Date;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
@@ -140,7 +138,6 @@ import java.util.zip.GZIPInputStream;
 public class StandardFlowSynchronizer implements FlowSynchronizer {
 
     private static final Logger logger = LoggerFactory.getLogger(StandardFlowSynchronizer.class);
-    public static final URL FLOW_XSD_RESOURCE = StandardFlowSynchronizer.class.getResource("/FlowConfiguration.xsd");
     private final StringEncryptor encryptor;
     private final boolean autoResumeState;
     private final NiFiProperties nifiProperties;
@@ -158,26 +155,12 @@ public class StandardFlowSynchronizer implements FlowSynchronizer {
             return true;
         }
 
-        final Document document = parseFlowBytes(dataFlow.getFlow());
-        final Element rootElement = document.getDocumentElement();
-
-        final Element rootGroupElement = (Element) rootElement.getElementsByTagName("rootGroup").item(0);
-        final FlowEncodingVersion encodingVersion = FlowEncodingVersion.parse(rootElement);
-        final ProcessGroupDTO rootGroupDto = FlowFromDOMFactory.getProcessGroup(null, rootGroupElement, null, encodingVersion);
-
-        final NodeList reportingTasks = rootElement.getElementsByTagName("reportingTask");
-        final ReportingTaskDTO reportingTaskDTO = reportingTasks.getLength() == 0 ? null : FlowFromDOMFactory.getReportingTask((Element)reportingTasks.item(0),null, encodingVersion);
-
-        final NodeList controllerServices = rootElement.getElementsByTagName("controllerService");
-        final ControllerServiceDTO controllerServiceDTO = controllerServices.getLength() == 0 ? null :
-            FlowFromDOMFactory.getControllerService((Element)controllerServices.item(0),null, encodingVersion);
-
-        return isEmpty(rootGroupDto) && isEmpty(reportingTaskDTO) && isEmpty(controllerServiceDTO);
+        return isFlowEmpty(dataFlow.getFlowDocument());
     }
 
     @Override
-    public void sync(final FlowController controller, final DataFlow proposedFlow, final StringEncryptor encryptor)
-            throws FlowSerializationException, UninheritableFlowException, FlowSynchronizationException, MissingBundleException {
+    public void sync(final FlowController controller, final DataFlow proposedFlow, final StringEncryptor encryptor, final FlowService flowService)
+            throws FlowSerializationException, UninheritableFlowException, FlowSynchronizationException {
 
         final FlowManager flowManager = controller.getFlowManager();
         final ProcessGroup root = flowManager.getRootGroup();
@@ -196,354 +179,345 @@ public class StandardFlowSynchronizer implements FlowSynchronizer {
         logger.debug("Synching FlowController with proposed flow: Controller is Already Synchronized = {}", flowAlreadySynchronized);
 
         // serialize controller state to bytes
-        final byte[] existingFlow;
-        final boolean existingFlowEmpty;
-        try {
-            if (flowAlreadySynchronized) {
-                existingFlow = toBytes(controller);
-                existingFlowEmpty = root.isEmpty()
-                    && flowManager.getAllReportingTasks().isEmpty()
-                    && flowManager.getAllControllerServices().isEmpty()
-                    && controller.getFlowRegistryClient().getRegistryIdentifiers().isEmpty()
-                    && controller.getFlowManager().getParameterContextManager().getParameterContexts().isEmpty();
-            } else {
-                existingFlow = readFlowFromDisk();
-                if (existingFlow == null || existingFlow.length == 0) {
-                    existingFlowEmpty = true;
-                } else {
-                    final Document document = parseFlowBytes(existingFlow);
-                    final Element rootElement = document.getDocumentElement();
-                    final FlowEncodingVersion encodingVersion = FlowEncodingVersion.parse(rootElement);
-
-                    logger.trace("Setting controller thread counts");
-                    final Integer maxThreadCount = getInteger(rootElement, "maxThreadCount");
-                    if (maxThreadCount == null) {
-                        controller.setMaxTimerDrivenThreadCount(getInt(rootElement, "maxTimerDrivenThreadCount"));
-                        controller.setMaxEventDrivenThreadCount(getInt(rootElement, "maxEventDrivenThreadCount"));
-                    } else {
-                        controller.setMaxTimerDrivenThreadCount(maxThreadCount * 2 / 3);
-                        controller.setMaxEventDrivenThreadCount(maxThreadCount / 3);
-                    }
+        final DataFlow existingDataFlow = getExistingDataFlow(controller);
+        boolean existingFlowEmpty = isFlowEmpty(existingDataFlow.getFlowDocument());
 
-                    final Element reportingTasksElement = DomUtils.getChild(rootElement, "reportingTasks");
-                    final List<Element> taskElements;
-                    if (reportingTasksElement == null) {
-                        taskElements = Collections.emptyList();
-                    } else {
-                        taskElements = DomUtils.getChildElementsByTagName(reportingTasksElement, "reportingTask");
-                    }
+        logger.trace("Parsing proposed flow bytes as DOM document");
+        final Document configuration = proposedFlow.getFlowDocument();
 
-                    final Element controllerServicesElement = DomUtils.getChild(rootElement, "controllerServices");
-                    final List<Element> unrootedControllerServiceElements;
-                    if (controllerServicesElement == null) {
-                        unrootedControllerServiceElements = Collections.emptyList();
-                    } else {
-                        unrootedControllerServiceElements = DomUtils.getChildElementsByTagName(controllerServicesElement, "controllerService");
-                    }
+        // check that the proposed flow is inheritable by the controller
+        boolean backupAndPurge = false;
+        if (existingFlowEmpty) {
+            logger.debug("Checking bundle compatibility");
 
-                    final boolean registriesPresent;
-                    final Element registriesElement = DomUtils.getChild(rootElement, "registries");
-                    if (registriesElement == null) {
-                        registriesPresent = false;
-                    } else {
-                        final List<Element> flowRegistryElems = DomUtils.getChildElementsByTagName(registriesElement, "flowRegistry");
-                        registriesPresent = !flowRegistryElems.isEmpty();
-                    }
+            final BundleCompatibilityCheck bundleCompatibilityCheck = new BundleCompatibilityCheck();
+            final FlowInheritability bundleInheritability = bundleCompatibilityCheck.checkInheritability(existingDataFlow, proposedFlow, controller);
+            if (!bundleInheritability.isInheritable()) {
+                throw new UninheritableFlowException("Proposed flow could not be inherited because it references one or more Bundles that are not available in this NiFi instance: "
+                    + bundleInheritability.getExplanation());
+            }
 
-                    final boolean parametersPresent;
-                    final Element parameterContextsElement = DomUtils.getChild(rootElement, "parameterContexts");
-                    if (parameterContextsElement == null) {
-                        parametersPresent = false;
-                    } else {
-                        final List<Element> contextList = DomUtils.getChildElementsByTagName(parameterContextsElement, "parameterContext");
-                        parametersPresent = !contextList.isEmpty();
-                    }
+            logger.debug("Bundle Compatibility check passed");
+        } else {
+            logger.debug("Checking flow inheritability");
+            final FlowInheritabilityCheck fingerprintCheck = new FlowFingerprintCheck();
+            final FlowInheritability inheritability = fingerprintCheck.checkInheritability(existingDataFlow, proposedFlow, controller);
 
-                    logger.trace("Parsing process group from DOM");
-                    final Element rootGroupElement = (Element) rootElement.getElementsByTagName("rootGroup").item(0);
-                    final ProcessGroupDTO rootGroupDto = FlowFromDOMFactory.getProcessGroup(null, rootGroupElement, encryptor, encodingVersion);
-                    existingFlowEmpty = taskElements.isEmpty()
-                        && unrootedControllerServiceElements.isEmpty()
-                        && isEmpty(rootGroupDto)
-                        && !registriesPresent
-                        && !parametersPresent;
-                    logger.debug("Existing Flow Empty = {}", existingFlowEmpty);
+            if (inheritability.isInheritable()) {
+                logger.debug("Proposed flow is inheritable");
+            } else {
+                if (controller.isInitialized()) {
+                    // Flow has already been initialized so cannot inherit the cluster flow as liberally.
+                    // Since the cluster's flow is not immediately inheritable, we must throw an UninheritableFlowException.
+                    throw new UninheritableFlowException("Proposed configuration is not inheritable by the flow controller because of flow differences: " + inheritability.getExplanation());
+                }
+
+                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 FlowInheritability connectionMissingInheritability = connectionMissingCheck.checkInheritability(existingDataFlow, proposedFlow, controller);
+                if (connectionMissingInheritability.isInheritable()) {
+                    backupAndPurge = true;
+                    existingFlowEmpty = true; // Consider the existing flow as being empty
+                    logger.debug("Proposed flow contains all connections that currently have data queued. Will backup existing flow and replace, provided all other checks pass");
+                } else {
+                    throw new UninheritableFlowException("Proposed flow is not inheritable by the flow controller and cannot completely replace the current flow due to: "
+                        + connectionMissingInheritability.getExplanation());
                 }
             }
-        } catch (final IOException e) {
-            throw new FlowSerializationException(e);
         }
 
-        logger.trace("Exporting snippets from controller");
-        final byte[] existingSnippets = controller.getSnippetManager().export();
-
-        logger.trace("Getting Authorizer fingerprint from controller");
+        logger.debug("Checking missing component inheritability");
+        final FlowInheritabilityCheck missingComponentsCheck = new MissingComponentsCheck();
+        final FlowInheritability componentInheritability = missingComponentsCheck.checkInheritability(existingDataFlow, proposedFlow, controller);
+        if (!componentInheritability.isInheritable()) {
+            throw new UninheritableFlowException("Proposed Flow is not inheritable by the flow controller because of differences in missing components: " + componentInheritability.getExplanation());
+        }
+        logger.debug("Missing Component Inheritability check passed");
 
-        final byte[] existingAuthFingerprint;
-        final ManagedAuthorizer managedAuthorizer;
+        logger.debug("Checking authorizer inheritability");
+        final FlowInheritabilityCheck authorizerCheck = new AuthorizerCheck();
+        final FlowInheritability authorizerInheritability = authorizerCheck.checkInheritability(existingDataFlow, proposedFlow, controller);
         final Authorizer authorizer = controller.getAuthorizer();
 
-        if (AuthorizerCapabilityDetection.isManagedAuthorizer(authorizer)) {
-            managedAuthorizer = (ManagedAuthorizer) authorizer;
-            existingAuthFingerprint = managedAuthorizer.getFingerprint().getBytes(StandardCharsets.UTF_8);
+        if (existingFlowEmpty) {
+            logger.debug("Existing flow is empty so will not check Authorizer inheritability. Authorizers will be forcibly inherited if necessary.");
         } else {
-            existingAuthFingerprint = null;
-            managedAuthorizer = null;
+            if (!controller.isInitialized() && authorizer instanceof ManagedAuthorizer) {
+                logger.debug("Authorizations are not inheritable, but Authorizer is a Managed Authorizer and the Controller has not yet been initialized, so it can be forcibly inherited.");
+            } else {
+                if (!authorizerInheritability.isInheritable() && authorizerInheritability.getExplanation() != null) {
+                    throw new UninheritableFlowException("Proposed Authorizer is not inheritable by the Flow Controller because NiFi has already started the dataflow " +
+                        "and Authorizer has differences: " + authorizerInheritability.getExplanation());
+                }
+
+                logger.debug("Authorizer inheritability check passed");
+            }
         }
 
-        final Set<String> missingComponents = new HashSet<>();
-        flowManager.getAllControllerServices().stream().filter(ComponentNode::isExtensionMissing).forEach(cs -> missingComponents.add(cs.getIdentifier()));
-        flowManager.getAllReportingTasks().stream().filter(ComponentNode::isExtensionMissing).forEach(r -> missingComponents.add(r.getIdentifier()));
-        root.findAllProcessors().stream().filter(AbstractComponentNode::isExtensionMissing).forEach(p -> missingComponents.add(p.getIdentifier()));
+        // attempt to sync controller with proposed flow
+        try {
+            if (backupAndPurge) {
+                logger.warn("Proposed flow cannot be directly inherited. However, all data that is queued in this instance is queued in a connection that exists in the Proposed flow. As a " +
+                    "result, the existing flow will be backed up and replaced with the proposed flow.");
+                final File backupFile = getFlowBackupFile();
 
-        final DataFlow existingDataFlow = new StandardDataFlow(existingFlow, existingSnippets, existingAuthFingerprint, missingComponents);
+                try {
+                    flowService.copyCurrentFlow(backupFile);
+                } catch (final IOException ioe) {
+                    throw new UninheritableFlowException("Could not inherit flow because failed to make a backup of existing flow to " + backupFile.getAbsolutePath(), ioe);
+                }
 
-        Document configuration = null;
+                logger.info("Successfully created backup of existing flow to {}. Will now purge local flow and inherit proposed flow", backupFile.getAbsolutePath());
+                controller.purge();
+            }
 
-        // check that the proposed flow is inheritable by the controller
-        try {
-            if (existingFlowEmpty) {
-                configuration = parseFlowBytes(proposedFlow.getFlow());
-                if (configuration != null) {
-                    logger.trace("Checking bundle compatibility");
-                    checkBundleCompatibility(configuration);
-                }
-            } else {
-                logger.trace("Checking flow inheritability");
-                final String problemInheritingFlow = checkFlowInheritability(existingDataFlow, proposedFlow, controller);
-                if (problemInheritingFlow != null) {
-                    throw new UninheritableFlowException("Proposed configuration is not inheritable by the flow controller because of flow differences: " + problemInheritingFlow);
-                }
+            if (!controller.isFlowSynchronized() && !existingFlowEmpty) {
+                updateThreadCounts(existingDataFlow.getFlowDocument().getDocumentElement(), controller);
             }
-        } catch (final FingerprintException fe) {
-            throw new FlowSerializationException("Failed to generate flow fingerprints", fe);
-        }
 
-        logger.trace("Checking missing component inheritability");
+            if (configuration != null) {
+                updateFlow(controller, configuration, existingDataFlow, existingFlowEmpty);
+            }
 
-        final String problemInheritingMissingComponents = checkMissingComponentsInheritability(existingDataFlow, proposedFlow);
-        if (problemInheritingMissingComponents != null) {
-            throw new UninheritableFlowException("Proposed Flow is not inheritable by the flow controller because of differences in missing components: " + problemInheritingMissingComponents);
-        }
+            inheritSnippets(controller, proposedFlow);
 
-        logger.trace("Checking authorizer inheritability");
+            // if auths are inheritable and we have a policy based authorizer, then inherit
+            if (authorizer instanceof ManagedAuthorizer) {
+                final ManagedAuthorizer managedAuthorizer = (ManagedAuthorizer) authorizer;
+                final String proposedAuthFingerprint = proposedFlow.getAuthorizerFingerprint() == null ? "" : new String(proposedFlow.getAuthorizerFingerprint(), StandardCharsets.UTF_8);
+
+                if (authorizerInheritability.isInheritable()) {
+                    logger.debug("Authorizations are inheritable. Will inherit from proposed fingerprint {}", proposedAuthFingerprint);
+                    managedAuthorizer.inheritFingerprint(proposedAuthFingerprint);
+                } else if (!Objects.equals(managedAuthorizer.getFingerprint(), proposedAuthFingerprint)) {
+                    // At this point, the flow is not inheritable, but we've made it this far. This can only happen if the existing flow is empty, so we can
+                    // just forcibly inherit the authorizations.
+                    logger.debug("Authorizations are not inheritable. Will force inheritance of proposed fingerprint {}", proposedAuthFingerprint);
+                    managedAuthorizer.forciblyInheritFingerprint(proposedAuthFingerprint);
+                }
+            }
 
-        final AuthorizerInheritability authInheritability = checkAuthorizerInheritability(authorizer, existingDataFlow, proposedFlow);
-        if (!authInheritability.isInheritable() && authInheritability.getReason() != null) {
-            throw new UninheritableFlowException("Proposed Authorizer is not inheritable by the flow controller because of Authorizer differences: " + authInheritability.getReason());
+            logger.debug("Finished syncing flows");
+        } catch (final Exception ex) {
+            throw new FlowSynchronizationException(ex);
         }
+    }
 
-        // create document by parsing proposed flow bytes
-        logger.trace("Parsing proposed flow bytes as DOM document");
-        if (configuration == null) {
-            configuration = parseFlowBytes(proposedFlow.getFlow());
+    private File getFlowBackupFile() {
+        final File flowConfigurationFile = nifiProperties.getFlowConfigurationFile();
+        final String baseFilename = StringUtils.substringBeforeLast(flowConfigurationFile.getName(), ".xml.gz");
+        final DateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd-HH-mm-ss");
+        final String timestamp = dateFormat.format(new Date());
+        final String backupFilename = baseFilename + "-" + timestamp + ".xml.gz";
+        final File backupFile = new File(flowConfigurationFile.getParentFile(), backupFilename);
+
+        if (!backupFile.getParentFile().exists() && !backupFile.getParentFile().mkdirs()) {
+            throw new UninheritableFlowException("Failed to backup existing flow because the configured directory for flow.xml.gz <" + backupFile.getParentFile().getAbsolutePath()
+                + "> does not exist and could not be created");
         }
 
-        // attempt to sync controller with proposed flow
-        try {
-            if (configuration != null) {
-                synchronized (configuration) {
-                    // get the root element
-                    final Element rootElement = (Element) configuration.getElementsByTagName("flowController").item(0);
-                    final FlowEncodingVersion encodingVersion = FlowEncodingVersion.parse(rootElement);
-
-                    // set controller config
-                    logger.trace("Updating flow config");
-                    final Integer maxThreadCount = getInteger(rootElement, "maxThreadCount");
-                    if (maxThreadCount == null) {
-                        controller.setMaxTimerDrivenThreadCount(getInt(rootElement, "maxTimerDrivenThreadCount"));
-                        controller.setMaxEventDrivenThreadCount(getInt(rootElement, "maxEventDrivenThreadCount"));
-                    } else {
-                        controller.setMaxTimerDrivenThreadCount(maxThreadCount * 2 / 3);
-                        controller.setMaxEventDrivenThreadCount(maxThreadCount / 3);
-                    }
+        return backupFile;
+    }
 
-                    // get the root group XML element
-                    final Element rootGroupElement = (Element) rootElement.getElementsByTagName("rootGroup").item(0);
-
-                    if (!flowAlreadySynchronized || existingFlowEmpty) {
-                        final Element registriesElement = DomUtils.getChild(rootElement, "registries");
-                        if (registriesElement != null) {
-                            final List<Element> flowRegistryElems = DomUtils.getChildElementsByTagName(registriesElement, "flowRegistry");
-                            for (final Element flowRegistryElement : flowRegistryElems) {
-                                final String registryId = getString(flowRegistryElement, "id");
-                                final String registryName = getString(flowRegistryElement, "name");
-                                final String registryUrl = getString(flowRegistryElement, "url");
-                                final String description = getString(flowRegistryElement, "description");
-
-                                final FlowRegistryClient client = controller.getFlowRegistryClient();
-                                client.addFlowRegistry(registryId, registryName, registryUrl, description);
-                            }
-                        }
+    private DataFlow getExistingDataFlow(final FlowController controller) {
+        final FlowManager flowManager = controller.getFlowManager();
+        final ProcessGroup root = flowManager.getRootGroup();
 
-                        final Element parameterContextsElement = DomUtils.getChild(rootElement, "parameterContexts");
-                        if (parameterContextsElement != null) {
-                            final List<Element> contextElements = DomUtils.getChildElementsByTagName(parameterContextsElement, "parameterContext");
-                            for (final Element contextElement : contextElements) {
-                                final ParameterContextDTO parameterContextDto = FlowFromDOMFactory.getParameterContext(contextElement, encryptor);
-                                createParameterContext(parameterContextDto, controller.getFlowManager());
-                            }
-                        }
-                    }
+        // Determine missing components
+        final Set<String> missingComponents = new HashSet<>();
+        flowManager.getAllControllerServices().stream().filter(ComponentNode::isExtensionMissing).forEach(cs -> missingComponents.add(cs.getIdentifier()));
+        flowManager.getAllReportingTasks().stream().filter(ComponentNode::isExtensionMissing).forEach(r -> missingComponents.add(r.getIdentifier()));
+        root.findAllProcessors().stream().filter(AbstractComponentNode::isExtensionMissing).forEach(p -> missingComponents.add(p.getIdentifier()));
 
-                    // if this controller isn't initialized or its empty, add the root group, otherwise update
-                    final ProcessGroup rootGroup;
-                    if (!flowAlreadySynchronized || existingFlowEmpty) {
-                        logger.trace("Adding root process group");
-                        rootGroup = addProcessGroup(controller, /* parent group */ null, rootGroupElement, encryptor, encodingVersion);
-                    } else {
-                        logger.trace("Updating root process group");
-                        rootGroup = updateProcessGroup(controller, /* parent group */ null, rootGroupElement, encryptor, encodingVersion);
-                    }
+        logger.trace("Exporting snippets from controller");
+        final byte[] existingSnippets = controller.getSnippetManager().export();
 
-                    rootGroup.findAllRemoteProcessGroups().forEach(RemoteProcessGroup::initialize);
-
-                    // If there are any Templates that do not exist in the Proposed Flow that do exist in the 'existing flow', we need
-                    // to ensure that we also add those to the appropriate Process Groups, so that we don't lose them.
-                    final Document existingFlowConfiguration = parseFlowBytes(existingFlow);
-                    if (existingFlowConfiguration != null) {
-                        final Element existingRootElement = (Element) existingFlowConfiguration.getElementsByTagName("flowController").item(0);
-                        if (existingRootElement != null) {
-                            final Element existingRootGroupElement = (Element) existingRootElement.getElementsByTagName("rootGroup").item(0);
-                            if (existingRootElement != null) {
-                                final FlowEncodingVersion existingEncodingVersion = FlowEncodingVersion.parse(existingFlowConfiguration.getDocumentElement());
-                                addLocalTemplates(existingRootGroupElement, rootGroup, existingEncodingVersion);
-                            }
-                        }
-                    }
+        final byte[] existingAuthFingerprint;
+        final Authorizer authorizer = controller.getAuthorizer();
+        if (AuthorizerCapabilityDetection.isManagedAuthorizer(authorizer)) {
+            final ManagedAuthorizer managedAuthorizer = (ManagedAuthorizer) authorizer;
+            existingAuthFingerprint = managedAuthorizer.getFingerprint().getBytes(StandardCharsets.UTF_8);
+        } else {
+            existingAuthFingerprint = null;
+        }
 
-                    // get all the reporting task elements
-                    final Element reportingTasksElement = DomUtils.getChild(rootElement, "reportingTasks");
-                    final List<Element> reportingTaskElements = new ArrayList<>();
-                    if (reportingTasksElement != null) {
-                        reportingTaskElements.addAll(DomUtils.getChildElementsByTagName(reportingTasksElement, "reportingTask"));
-                    }
+        // serialize controller state to bytes
+        final byte[] existingFlow;
+        try {
+            if (controller.isFlowSynchronized()) {
+                existingFlow = toBytes(controller);
+                return new StandardDataFlow(existingFlow, existingSnippets, existingAuthFingerprint, missingComponents);
+            } else {
+                existingFlow = readFlowFromDisk();
+                if (existingFlow == null || existingFlow.length == 0) {
+                    return new StandardDataFlow(existingFlow, existingSnippets, existingAuthFingerprint, missingComponents);
+                } else {
+                    return new StandardDataFlow(existingFlow, existingSnippets, existingAuthFingerprint, missingComponents);
+                }
+            }
+        } catch (final IOException e) {
+            throw new FlowSerializationException(e);
+        }
+    }
 
-                    // get/create all the reporting task nodes and DTOs, but don't apply their scheduled state yet
-                    final Map<ReportingTaskNode, ReportingTaskDTO> reportingTaskNodesToDTOs = new HashMap<>();
-                    for (final Element taskElement : reportingTaskElements) {
-                        final ReportingTaskDTO dto = FlowFromDOMFactory.getReportingTask(taskElement, encryptor, encodingVersion);
-                        final ReportingTaskNode reportingTask = getOrCreateReportingTask(controller, dto, flowAlreadySynchronized, existingFlowEmpty);
-                        reportingTaskNodesToDTOs.put(reportingTask, dto);
-                    }
+    private void inheritSnippets(final FlowController controller, final DataFlow proposedFlow) {
+        // clear the snippets that are currently in memory
+        logger.trace("Clearing existing snippets");
+        final SnippetManager snippetManager = controller.getSnippetManager();
+        snippetManager.clear();
 
-                    final Element controllerServicesElement = DomUtils.getChild(rootElement, "controllerServices");
-                    if (controllerServicesElement != null) {
-                        final List<Element> serviceElements = DomUtils.getChildElementsByTagName(controllerServicesElement, "controllerService");
-
-                        if (!flowAlreadySynchronized || existingFlowEmpty) {
-                            // If the encoding version is null, we are loading a flow from NiFi 0.x, where Controller
-                            // Services could not be scoped by Process Group. As a result, we want to move the Process Groups
-                            // to the root Group. Otherwise, we want to use a null group, which indicates a Controller-level
-                            // Controller Service.
-                            final ProcessGroup group = (encodingVersion == null) ? rootGroup : null;
-                            final Map<ControllerServiceNode, Element> controllerServices = ControllerServiceLoader.loadControllerServices(
-                                serviceElements, controller, group, encryptor, encodingVersion);
-
-                            // If we are moving controller services to the root group we also need to see if any reporting tasks
-                            // reference them, and if so we need to clone the CS and update the reporting task reference
-                            if (group != null) {
-                                // find all the controller service ids referenced by reporting tasks
-                                final Set<String> controllerServicesInReportingTasks = reportingTaskNodesToDTOs.keySet().stream()
-                                        .flatMap(r -> r.getEffectivePropertyValues().entrySet().stream())
-                                        .filter(e -> e.getKey().getControllerServiceDefinition() != null)
-                                        .map(Map.Entry::getValue)
-                                        .collect(Collectors.toSet());
-
-                                // find the controller service nodes for each id referenced by a reporting task
-                                final Set<ControllerServiceNode> controllerServicesToClone = controllerServices.keySet().stream()
-                                        .filter(cs -> controllerServicesInReportingTasks.contains(cs.getIdentifier()))
-                                        .collect(Collectors.toSet());
-
-                                // clone the controller services and map the original id to the clone
-                                final Map<String, ControllerServiceNode> controllerServiceMapping = new HashMap<>();
-                                for (ControllerServiceNode controllerService : controllerServicesToClone) {
-                                    final ControllerServiceNode clone = ControllerServiceLoader.cloneControllerService(controller, controllerService);
-                                    flowManager.addRootControllerService(clone);
-                                    controllerServiceMapping.put(controllerService.getIdentifier(), clone);
-                                }
-
-                                // update the reporting tasks to reference the cloned controller services
-                                updateReportingTaskControllerServices(reportingTaskNodesToDTOs.keySet(), controllerServiceMapping);
-
-                                // enable all the cloned controller services
-                                ControllerServiceLoader.enableControllerServices(controllerServiceMapping.values(), controller, autoResumeState);
-                            }
+        // if proposed flow has any snippets, load them
+        logger.trace("Loading proposed snippets");
+        final byte[] proposedSnippets = proposedFlow.getSnippets();
+        if (proposedSnippets != null && proposedSnippets.length > 0) {
+            for (final StandardSnippet snippet : SnippetManager.parseBytes(proposedSnippets)) {
+                snippetManager.addSnippet(snippet);
+            }
+        }
+    }
 
-                            // enable all the original controller services
-                            ControllerServiceLoader.enableControllerServices(controllerServices, controller, encryptor, autoResumeState, encodingVersion);
-                        }
-                    }
+    private void updateFlow(final FlowController controller, final Document configuration, final DataFlow existingFlow, final boolean existingFlowEmpty) throws ReportingTaskInstantiationException {
+        final boolean flowAlreadySynchronized = controller.isFlowSynchronized();
+        final FlowManager flowManager = controller.getFlowManager();
 
-                    scaleRootGroup(rootGroup, encodingVersion);
+        // get the root element
+        final Element rootElement = (Element) configuration.getElementsByTagName("flowController").item(0);
+        final FlowEncodingVersion encodingVersion = FlowEncodingVersion.parse(rootElement);
 
-                    // now that controller services are loaded and enabled we can apply the scheduled state to each reporting task
-                    for (Map.Entry<ReportingTaskNode, ReportingTaskDTO> entry : reportingTaskNodesToDTOs.entrySet()) {
-                        applyReportingTaskScheduleState(controller, entry.getValue(), entry.getKey(), flowAlreadySynchronized, existingFlowEmpty);
-                    }
-                }
-            }
+        // set controller config
+        logger.trace("Updating flow config");
+        final Integer maxThreadCount = getInteger(rootElement, "maxThreadCount");
+        if (maxThreadCount == null) {
+            controller.setMaxTimerDrivenThreadCount(getInt(rootElement, "maxTimerDrivenThreadCount"));
+            controller.setMaxEventDrivenThreadCount(getInt(rootElement, "maxEventDrivenThreadCount"));
+        } else {
+            controller.setMaxTimerDrivenThreadCount(maxThreadCount * 2 / 3);
+            controller.setMaxEventDrivenThreadCount(maxThreadCount / 3);
+        }
 
-            // clear the snippets that are currently in memory
-            logger.trace("Clearing existing snippets");
-            final SnippetManager snippetManager = controller.getSnippetManager();
-            snippetManager.clear();
+        // get the root group XML element
+        final Element rootGroupElement = (Element) rootElement.getElementsByTagName("rootGroup").item(0);
 
-            // if proposed flow has any snippets, load them
-            logger.trace("Loading proposed snippets");
-            final byte[] proposedSnippets = proposedFlow.getSnippets();
-            if (proposedSnippets != null && proposedSnippets.length > 0) {
-                for (final StandardSnippet snippet : SnippetManager.parseBytes(proposedSnippets)) {
-                    snippetManager.addSnippet(snippet);
+        // if this controller isn't initialized or its empty, add the root group, otherwise update
+        final ProcessGroup rootGroup;
+        if (!flowAlreadySynchronized || existingFlowEmpty) {
+            final Element registriesElement = DomUtils.getChild(rootElement, "registries");
+            if (registriesElement != null) {
+                final List<Element> flowRegistryElems = DomUtils.getChildElementsByTagName(registriesElement, "flowRegistry");
+                for (final Element flowRegistryElement : flowRegistryElems) {
+                    final String registryId = getString(flowRegistryElement, "id");
+                    final String registryName = getString(flowRegistryElement, "name");
+                    final String registryUrl = getString(flowRegistryElement, "url");
+                    final String description = getString(flowRegistryElement, "description");
+
+                    final FlowRegistryClient client = controller.getFlowRegistryClient();
+                    client.addFlowRegistry(registryId, registryName, registryUrl, description);
                 }
             }
 
-            // if auths are inheritable and we have a policy based authorizer, then inherit
-            if (authInheritability.isInheritable() && managedAuthorizer != null) {
-                logger.trace("Inheriting authorizations");
-                final String proposedAuthFingerprint = new String(proposedFlow.getAuthorizerFingerprint(), StandardCharsets.UTF_8);
-                managedAuthorizer.inheritFingerprint(proposedAuthFingerprint);
+            final Element parameterContextsElement = DomUtils.getChild(rootElement, "parameterContexts");
+            if (parameterContextsElement != null) {
+                final List<Element> contextElements = DomUtils.getChildElementsByTagName(parameterContextsElement, "parameterContext");
+                for (final Element contextElement : contextElements) {
+                    final ParameterContextDTO parameterContextDto = FlowFromDOMFactory.getParameterContext(contextElement, encryptor);
+                    createParameterContext(parameterContextDto, controller.getFlowManager());
+                }
             }
 
-            logger.debug("Finished syncing flows");
-        } catch (final Exception ex) {
-            throw new FlowSynchronizationException(ex);
+            logger.trace("Adding root process group");
+            rootGroup = addProcessGroup(controller, /* parent group */ null, rootGroupElement, encryptor, encodingVersion);
+        } else {
+            logger.trace("Updating root process group");
+            rootGroup = updateProcessGroup(controller, /* parent group */ null, rootGroupElement, encryptor, encodingVersion);
+        }
+
+        rootGroup.findAllRemoteProcessGroups().forEach(RemoteProcessGroup::initialize);
+
+        // If there are any Templates that do not exist in the Proposed Flow that do exist in the 'existing flow', we need
+        // to ensure that we also add those to the appropriate Process Groups, so that we don't lose them.
+        if (!existingFlowEmpty) {
+            final Document existingFlowConfiguration = existingFlow.getFlowDocument();
+            if (existingFlowConfiguration != null) {
+                final Element existingRootElement = (Element) existingFlowConfiguration.getElementsByTagName("flowController").item(0);
+                if (existingRootElement != null) {
+                    final Element existingRootGroupElement = (Element) existingRootElement.getElementsByTagName("rootGroup").item(0);
+                    if (existingRootElement != null) {
+                        addLocalTemplates(existingRootGroupElement, rootGroup);
+                    }
+                }
+            }
         }
-    }
 
-    private void checkBundleCompatibility(final Document configuration) {
-        final NodeList bundleNodes = configuration.getElementsByTagName("bundle");
-        for (int i = 0; i < bundleNodes.getLength(); i++) {
-            final Node bundleNode = bundleNodes.item(i);
-            if (bundleNode instanceof Element) {
-                final Element bundleElement = (Element) bundleNode;
-
-                final Node componentNode = bundleElement.getParentNode();
-                if (componentNode instanceof Element) {
-                    final Element componentElement = (Element) componentNode;
-                    if (!withinTemplate(componentElement)) {
-                        final String componentType = DomUtils.getChildText(componentElement, "class");
-                        try {
-                            BundleUtils.getBundle(extensionManager, componentType, FlowFromDOMFactory.getBundle(bundleElement));
-                        } catch (IllegalStateException e) {
-                            throw new MissingBundleException(e.getMessage(), e);
-                        }
+        // get all the reporting task elements
+        final Element reportingTasksElement = DomUtils.getChild(rootElement, "reportingTasks");
+        final List<Element> reportingTaskElements = new ArrayList<>();
+        if (reportingTasksElement != null) {
+            reportingTaskElements.addAll(DomUtils.getChildElementsByTagName(reportingTasksElement, "reportingTask"));
+        }
+
+        // get/create all the reporting task nodes and DTOs, but don't apply their scheduled state yet
+        final Map<ReportingTaskNode, ReportingTaskDTO> reportingTaskNodesToDTOs = new HashMap<>();
+        for (final Element taskElement : reportingTaskElements) {
+            final ReportingTaskDTO dto = FlowFromDOMFactory.getReportingTask(taskElement, encryptor, encodingVersion);
+            final ReportingTaskNode reportingTask = getOrCreateReportingTask(controller, dto, flowAlreadySynchronized, existingFlowEmpty);
+            reportingTaskNodesToDTOs.put(reportingTask, dto);
+        }
+
+        final Element controllerServicesElement = DomUtils.getChild(rootElement, "controllerServices");
+        if (controllerServicesElement != null) {
+            final List<Element> serviceElements = DomUtils.getChildElementsByTagName(controllerServicesElement, "controllerService");
+
+            if (!flowAlreadySynchronized || existingFlowEmpty) {
+                // If the encoding version is null, we are loading a flow from NiFi 0.x, where Controller
+                // Services could not be scoped by Process Group. As a result, we want to move the Process Groups
+                // to the root Group. Otherwise, we want to use a null group, which indicates a Controller-level
+                // Controller Service.
+                final ProcessGroup group = (encodingVersion == null) ? rootGroup : null;
+                final Map<ControllerServiceNode, Element> controllerServices = ControllerServiceLoader.loadControllerServices(
+                    serviceElements, controller, group, encryptor, encodingVersion);
+
+                // If we are moving controller services to the root group we also need to see if any reporting tasks
+                // reference them, and if so we need to clone the CS and update the reporting task reference
+                if (group != null) {
+                    // find all the controller service ids referenced by reporting tasks
+                    final Set<String> controllerServicesInReportingTasks = reportingTaskNodesToDTOs.keySet().stream()
+                        .flatMap(r -> r.getEffectivePropertyValues().entrySet().stream())
+                        .filter(e -> e.getKey().getControllerServiceDefinition() != null)
+                        .map(Map.Entry::getValue)
+                        .collect(Collectors.toSet());
+
+                    // find the controller service nodes for each id referenced by a reporting task
+                    final Set<ControllerServiceNode> controllerServicesToClone = controllerServices.keySet().stream()
+                        .filter(cs -> controllerServicesInReportingTasks.contains(cs.getIdentifier()))
+                        .collect(Collectors.toSet());
+
+                    // clone the controller services and map the original id to the clone
+                    final Map<String, ControllerServiceNode> controllerServiceMapping = new HashMap<>();
+                    for (ControllerServiceNode controllerService : controllerServicesToClone) {
+                        final ControllerServiceNode clone = ControllerServiceLoader.cloneControllerService(controller, controllerService);
+                        flowManager.addRootControllerService(clone);
+                        controllerServiceMapping.put(controllerService.getIdentifier(), clone);
                     }
+
+                    // update the reporting tasks to reference the cloned controller services
+                    updateReportingTaskControllerServices(reportingTaskNodesToDTOs.keySet(), controllerServiceMapping);
+
+                    // enable all the cloned controller services
+                    ControllerServiceLoader.enableControllerServices(controllerServiceMapping.values(), controller, autoResumeState);
                 }
+
+                // enable all the original controller services
+                ControllerServiceLoader.enableControllerServices(controllerServices, controller, encryptor, autoResumeState, encodingVersion);
             }
         }
-    }
 
-    private boolean withinTemplate(final Element element) {
-        if ("template".equals(element.getTagName())) {
-            return true;
-        } else {
-            final Node parentNode = element.getParentNode();
-            if (parentNode instanceof Element) {
-                return withinTemplate((Element) parentNode);
-            } else {
-                return false;
-            }
+        scaleRootGroup(rootGroup, encodingVersion);
+
+        // now that controller services are loaded and enabled we can apply the scheduled state to each reporting task
+        for (Map.Entry<ReportingTaskNode, ReportingTaskDTO> entry : reportingTaskNodesToDTOs.entrySet()) {
+            applyReportingTaskScheduleState(controller, entry.getValue(), entry.getKey(), flowAlreadySynchronized, existingFlowEmpty);
         }
     }
 
@@ -594,7 +568,7 @@ public class StandardFlowSynchronizer implements FlowSynchronizer {
         }
     }
 
-    private void addLocalTemplates(final Element processGroupElement, final ProcessGroup processGroup, final FlowEncodingVersion encodingVersion) {
+    private void addLocalTemplates(final Element processGroupElement, final ProcessGroup processGroup) {
         // Replace the templates with those from the proposed flow
         final List<Element> templateNodeList = getChildrenByTagName(processGroupElement, "template");
         if (templateNodeList != null) {
@@ -613,7 +587,7 @@ public class StandardFlowSynchronizer implements FlowSynchronizer {
         for (final Element childGroupElement : childGroupElements) {
             final String childGroupId = getString(childGroupElement, "id");
             final ProcessGroup childGroup = processGroup.getProcessGroup(childGroupId);
-            addLocalTemplates(childGroupElement, childGroup, encodingVersion);
+            addLocalTemplates(childGroupElement, childGroup);
         }
     }
 
@@ -624,6 +598,69 @@ public class StandardFlowSynchronizer implements FlowSynchronizer {
         }
     }
 
+    private void updateThreadCounts(final Element rootElement, final FlowController controller) {
+        logger.trace("Setting controller thread counts");
+        final Integer maxThreadCount = getInteger(rootElement, "maxThreadCount");
+        if (maxThreadCount == null) {
+            controller.setMaxTimerDrivenThreadCount(getInt(rootElement, "maxTimerDrivenThreadCount"));
+            controller.setMaxEventDrivenThreadCount(getInt(rootElement, "maxEventDrivenThreadCount"));
+        } else {
+            controller.setMaxTimerDrivenThreadCount(maxThreadCount * 2 / 3);
+            controller.setMaxEventDrivenThreadCount(maxThreadCount / 3);
+        }
+    }
+
+
+    private static boolean isFlowEmpty(final Document flowDocument) {
+        if (flowDocument == null) {
+            return true;
+        }
+
+        final Element rootElement = flowDocument.getDocumentElement();
+        if (rootElement == null) {
+            return true;
+        }
+
+        final FlowEncodingVersion encodingVersion = FlowEncodingVersion.parse(rootElement);
+
+        final Element reportingTasksElement = DomUtils.getChild(rootElement, "reportingTasks");
+        if (reportingTasksElement != null) {
+            final List<Element> taskElements = DomUtils.getChildElementsByTagName(reportingTasksElement, "reportingTask");
+            if (!taskElements.isEmpty()) {
+                return false;
+            }
+        }
+
+        final Element controllerServicesElement = DomUtils.getChild(rootElement, "controllerServices");
+        if (controllerServicesElement != null) {
+            final List<Element> unrootedControllerServiceElements = DomUtils.getChildElementsByTagName(controllerServicesElement, "controllerService");
+            if (!unrootedControllerServiceElements.isEmpty()) {
+                return false;
+            }
+        }
+
+        final Element registriesElement = DomUtils.getChild(rootElement, "registries");
+        if (registriesElement != null) {
+            final List<Element> flowRegistryElems = DomUtils.getChildElementsByTagName(registriesElement, "flowRegistry");
+            if (!flowRegistryElems.isEmpty()) {
+                return false;
+            }
+        }
+
+        final Element parameterContextsElement = DomUtils.getChild(rootElement, "parameterContexts");
+        if (parameterContextsElement != null) {
+            final List<Element> contextList = DomUtils.getChildElementsByTagName(parameterContextsElement, "parameterContext");
+            if (!contextList.isEmpty()) {
+                return false;
+            }
+        }
+
+        logger.trace("Parsing process group from DOM");
+        final Element rootGroupElement = (Element) rootElement.getElementsByTagName("rootGroup").item(0);
+        final ProcessGroupDTO rootGroupDto = FlowFromDOMFactory.getProcessGroup(null, rootGroupElement, null, encodingVersion);
+        return isEmpty(rootGroupDto);
+    }
+
     private static boolean isEmpty(final ProcessGroupDTO dto) {
         if (dto == null) {
             return true;
@@ -640,44 +677,13 @@ public class StandardFlowSynchronizer implements FlowSynchronizer {
                 && CollectionUtils.isEmpty(contents.getConnections())
                 && CollectionUtils.isEmpty(contents.getFunnels())
                 && CollectionUtils.isEmpty(contents.getLabels())
+                && CollectionUtils.isEmpty(contents.getInputPorts())
                 && CollectionUtils.isEmpty(contents.getOutputPorts())
                 && CollectionUtils.isEmpty(contents.getProcessGroups())
-                && CollectionUtils.isEmpty(contents.getProcessors())
                 && CollectionUtils.isEmpty(contents.getRemoteProcessGroups())
                 && parameterContextId == null;
     }
 
-    private static boolean isEmpty(final ReportingTaskDTO reportingTaskDTO){
-
-       return reportingTaskDTO == null || StringUtils.isEmpty(reportingTaskDTO.getName()) ;
-
-    }
-
-    private static boolean isEmpty(final ControllerServiceDTO controllerServiceDTO){
-
-        return controllerServiceDTO == null || StringUtils.isEmpty(controllerServiceDTO.getName());
-
-    }
-
-    private static Document parseFlowBytes(final byte[] flow) throws FlowSerializationException {
-        // create document by parsing proposed flow bytes
-        try {
-            // create validating document builder
-            final SchemaFactory schemaFactory = SchemaFactory.newInstance(XMLConstants.W3C_XML_SCHEMA_NS_URI);
-            final Schema schema = schemaFactory.newSchema(FLOW_XSD_RESOURCE);
-            final DocumentBuilderFactory docFactory = DocumentBuilderFactory.newInstance();
-            docFactory.setNamespaceAware(true);
-            docFactory.setSchema(schema);
-
-            final DocumentBuilder docBuilder = docFactory.newDocumentBuilder();
-            docBuilder.setErrorHandler(new LoggingXmlParserErrorHandler("Flow Configuration", logger));
-
-            // parse flow
-            return (flow == null || flow.length == 0) ? null : docBuilder.parse(new ByteArrayInputStream(flow));
-        } catch (final SAXException | ParserConfigurationException | IOException ex) {
-            throw new FlowSerializationException(ex);
-        }
-    }
 
     private byte[] readFlowFromDisk() throws IOException {
         final Path flowPath = nifiProperties.getFlowConfigurationFile().toPath();
@@ -1274,24 +1280,33 @@ public class StandardFlowSynchronizer implements FlowSynchronizer {
             processGroup.setParameterContext(parameterContext);
         }
 
-        // Set the variables for the variable registry
-        final Map<String, String> variables = new HashMap<>();
-        final List<Element> variableElements = getChildrenByTagName(processGroupElement, "variable");
-        for (final Element variableElement : variableElements) {
-            final String variableName = variableElement.getAttribute("name");
-            final String variableValue = variableElement.getAttribute("value");
-            if (variableName == null || variableValue == null) {
-                continue;
-            }
+        addVariables(processGroupElement, processGroup);
+        addVersionControlInfo(processGroup, processGroupDTO, controller);
+        addControllerServices(processGroupElement, processGroup, controller, encodingVersion);
+        addProcessors(processGroupElement, processGroup, controller, encodingVersion);
+        addInputPorts(processGroupElement, processGroup, controller);
+        addOutputPorts(processGroupElement, processGroup, controller);
+        addFunnels(processGroupElement, processGroup, controller);
+        addLabels(processGroupElement, processGroup, controller);
+        addNestedProcessGroups(processGroupElement, processGroup, controller, encodingVersion);
+        addRemoteProcessGroups(processGroupElement, processGroup, controller);
+        addConnections(processGroupElement, processGroup, controller);
+        addTemplates(processGroupElement, processGroup);
 
-            variables.put(variableName, variableValue);
-        }
+        return processGroup;
+    }
 
-        processGroup.setVariables(variables);
+    private void addNestedProcessGroups(final Element processGroupElement, final ProcessGroup processGroup, final FlowController flowController, final FlowEncodingVersion encodingVersion) {
+        final List<Element> nestedProcessGroupNodeList = getChildrenByTagName(processGroupElement, "processGroup");
+        for (final Element nestedProcessGroupElement : nestedProcessGroupNodeList) {
+            addProcessGroup(flowController, processGroup, nestedProcessGroupElement, encryptor, encodingVersion);
+        }
+    }
 
+    private void addVersionControlInfo(final ProcessGroup processGroup, final ProcessGroupDTO processGroupDTO, final FlowController flowController) {
         final VersionControlInformationDTO versionControlInfoDto = processGroupDTO.getVersionControlInformation();
         if (versionControlInfoDto != null) {
-            final FlowRegistry flowRegistry = controller.getFlowRegistryClient().getFlowRegistry(versionControlInfoDto.getRegistryId());
+            final FlowRegistry flowRegistry = flowController.getFlowRegistryClient().getFlowRegistry(versionControlInfoDto.getRegistryId());
             final String registryName = flowRegistry == null ? versionControlInfoDto.getRegistryId() : flowRegistry.getName();
 
             versionControlInfoDto.setState(VersionedFlowState.SYNC_FAILURE.name());
@@ -1303,15 +1318,33 @@ public class StandardFlowSynchronizer implements FlowSynchronizer {
             // pass empty map for the version control mapping because the VersionedComponentId has already been set on the components
             processGroup.setVersionControlInformation(versionControlInformation, Collections.emptyMap());
         }
+    }
 
-        // Add Controller Services
+    private void addVariables(final Element processGroupElement, final ProcessGroup processGroup) {
+        final Map<String, String> variables = new HashMap<>();
+        final List<Element> variableElements = getChildrenByTagName(processGroupElement, "variable");
+        for (final Element variableElement : variableElements) {
+            final String variableName = variableElement.getAttribute("name");
+            final String variableValue = variableElement.getAttribute("value");
+            if (variableName == null || variableValue == null) {
+                continue;
+            }
+
+            variables.put(variableName, variableValue);
+        }
+
+        processGroup.setVariables(variables);
+    }
+
+    private void addControllerServices(final Element processGroupElement, final ProcessGroup processGroup, final FlowController flowController, final FlowEncodingVersion encodingVersion) {
         final List<Element> serviceNodeList = getChildrenByTagName(processGroupElement, "controllerService");
         if (!serviceNodeList.isEmpty()) {
-            final Map<ControllerServiceNode, Element> controllerServices = ControllerServiceLoader.loadControllerServices(serviceNodeList, controller, processGroup, encryptor, encodingVersion);
-            ControllerServiceLoader.enableControllerServices(controllerServices, controller, encryptor, autoResumeState, encodingVersion);
+            final Map<ControllerServiceNode, Element> controllerServices = ControllerServiceLoader.loadControllerServices(serviceNodeList, flowController, processGroup, encryptor, encodingVersion);
+            ControllerServiceLoader.enableControllerServices(controllerServices, flowController, encryptor, autoResumeState, encodingVersion);
         }
+    }
 
-        // add processors
+    private void addProcessors(final Element processGroupElement, final ProcessGroup processGroup, final FlowController flowController, final FlowEncodingVersion encodingVersion) {
         final List<Element> processorNodeList = getChildrenByTagName(processGroupElement, "processor");
         for (final Element processorElement : processorNodeList) {
             final ProcessorDTO processorDTO = FlowFromDOMFactory.getProcessor(processorElement, encryptor, encodingVersion);
@@ -1328,13 +1361,15 @@ public class StandardFlowSynchronizer implements FlowSynchronizer {
                 }
             }
 
-            final ProcessorNode procNode = flowManager.createProcessor(processorDTO.getType(), processorDTO.getId(), coordinate, false);
+            final ProcessorNode procNode = flowController.getFlowManager().createProcessor(processorDTO.getType(), processorDTO.getId(), coordinate, false);
             procNode.setVersionedComponentId(processorDTO.getVersionedComponentId());
             processGroup.addProcessor(procNode);
-            updateProcessor(procNode, processorDTO, processGroup, controller);
+            updateProcessor(procNode, processorDTO, processGroup, flowController);
         }
+    }
 
-        // add input ports
+    private void addInputPorts(final Element processGroupElement, final ProcessGroup processGroup, final FlowController flowController) {
+        final FlowManager flowManager = flowController.getFlowManager();
         final List<Element> inputPortNodeList = getChildrenByTagName(processGroupElement, "inputPort");
         for (final Element inputPortElement : inputPortNodeList) {
             final PortDTO portDTO = FlowFromDOMFactory.getPort(inputPortElement);
@@ -1373,13 +1408,15 @@ public class StandardFlowSynchronizer implements FlowSynchronizer {
 
             final ScheduledState scheduledState = ScheduledState.valueOf(portDTO.getState());
             if (ScheduledState.RUNNING.equals(scheduledState)) {
-                controller.startConnectable(port);
+                flowController.startConnectable(port);
             } else if (ScheduledState.DISABLED.equals(scheduledState)) {
                 processGroup.disableInputPort(port);
             }
         }
+    }
 
-        // add output ports
+    private void addOutputPorts(final Element processGroupElement, final ProcessGroup processGroup, final FlowController flowController) {
+        final FlowManager flowManager = flowController.getFlowManager();
         final List<Element> outputPortNodeList = getChildrenByTagName(processGroupElement, "outputPort");
         for (final Element outputPortElement : outputPortNodeList) {
             final PortDTO portDTO = FlowFromDOMFactory.getPort(outputPortElement);
@@ -1418,17 +1455,18 @@ public class StandardFlowSynchronizer implements FlowSynchronizer {
 
             final ScheduledState scheduledState = ScheduledState.valueOf(portDTO.getState());
             if (ScheduledState.RUNNING.equals(scheduledState)) {
-                controller.startConnectable(port);
+                flowController.startConnectable(port);
             } else if (ScheduledState.DISABLED.equals(scheduledState)) {
                 processGroup.disableOutputPort(port);
             }
         }
+    }
 
-        // add funnels
+    private void addFunnels(final Element processGroupElement, final ProcessGroup processGroup, final FlowController controller) {
         final List<Element> funnelNodeList = getChildrenByTagName(processGroupElement, "funnel");
         for (final Element funnelElement : funnelNodeList) {
             final FunnelDTO funnelDTO = FlowFromDOMFactory.getFunnel(funnelElement);
-            final Funnel funnel = flowManager.createFunnel(funnelDTO.getId());
+            final Funnel funnel = controller.getFlowManager().createFunnel(funnelDTO.getId());
             funnel.setVersionedComponentId(funnelDTO.getVersionedComponentId());
             funnel.setPosition(toPosition(funnelDTO.getPosition()));
 
@@ -1438,12 +1476,13 @@ public class StandardFlowSynchronizer implements FlowSynchronizer {
             processGroup.addFunnel(funnel, false);
             controller.startConnectable(funnel);
         }
+    }
 
-        // add labels
+    private void addLabels(final Element processGroupElement, final ProcessGroup processGroup, final FlowController controller) {
         final List<Element> labelNodeList = getChildrenByTagName(processGroupElement, "label");
         for (final Element labelElement : labelNodeList) {
             final LabelDTO labelDTO = FlowFromDOMFactory.getLabel(labelElement);
-            final Label label = flowManager.createLabel(labelDTO.getId(), labelDTO.getLabel());
+            final Label label = controller.getFlowManager().createLabel(labelDTO.getId(), labelDTO.getLabel());
             label.setVersionedComponentId(labelDTO.getVersionedComponentId());
             label.setStyle(labelDTO.getStyle());
 
@@ -1451,18 +1490,13 @@ public class StandardFlowSynchronizer implements FlowSynchronizer {
             label.setSize(new Size(labelDTO.getWidth(), labelDTO.getHeight()));
             processGroup.addLabel(label);
         }
+    }
 
-        // add nested process groups (recursively)
-        final List<Element> nestedProcessGroupNodeList = getChildrenByTagName(processGroupElement, "processGroup");
-        for (final Element nestedProcessGroupElement : nestedProcessGroupNodeList) {
-            addProcessGroup(controller, processGroup, nestedProcessGroupElement, encryptor, encodingVersion);
-        }
-
-        // add remote process group
+    private void addRemoteProcessGroups(final Element processGroupElement, final ProcessGroup processGroup, final FlowController controller) {
         final List<Element> remoteProcessGroupNodeList = getChildrenByTagName(processGroupElement, "remoteProcessGroup");
         for (final Element remoteProcessGroupElement : remoteProcessGroupNodeList) {
             final RemoteProcessGroupDTO remoteGroupDto = FlowFromDOMFactory.getRemoteProcessGroup(remoteProcessGroupElement, encryptor);
-            final RemoteProcessGroup remoteGroup = flowManager.createRemoteProcessGroup(remoteGroupDto.getId(), remoteGroupDto.getTargetUris());
+            final RemoteProcessGroup remoteGroup = controller.getFlowManager().createRemoteProcessGroup(remoteGroupDto.getId(), remoteGroupDto.getTargetUris());
             remoteGroup.setVersionedComponentId(remoteGroupDto.getVersionedComponentId());
             remoteGroup.setComments(remoteGroupDto.getComments());
             remoteGroup.setPosition(toPosition(remoteGroupDto.getPosition()));
@@ -1530,8 +1564,11 @@ public class StandardFlowSynchronizer implements FlowSynchronizer {
                 }
             }
         }
+    }
+
+    private void addConnections(final Element processGroupElement, final ProcessGroup processGroup, final FlowController controller) {
+        final FlowManager flowManager = controller.getFlowManager();
 
-        // add connections
         final List<Element> connectionNodeList = getChildrenByTagName(processGroupElement, "connection");
         for (final Element connectionElement : connectionNodeList) {
             final ConnectionDTO dto = FlowFromDOMFactory.getConnection(connectionElement);
@@ -1626,176 +1663,17 @@ public class StandardFlowSynchronizer implements FlowSynchronizer {
 
             processGroup.addConnection(connection);
         }
+    }
 
+    private void addTemplates(final Element processGroupElement, final ProcessGroup processGroup) {
         final List<Element> templateNodeList = getChildrenByTagName(processGroupElement, "template");
         for (final Element templateNode : templateNodeList) {
             final TemplateDTO templateDTO = TemplateUtils.parseDto(templateNode);
             final Template template = new Template(templateDTO);
             processGroup.addTemplate(template);
         }
-
-        return processGroup;
-    }
-
-    public String checkMissingComponentsInheritability(final DataFlow existingFlow, final DataFlow proposedFlow) {
-        if (existingFlow == null) {
-            return null;  // no existing flow, so equivalent to proposed flow
-        }
-
-        final Set<String> existingMissingComponents = new HashSet<>(existingFlow.getMissingComponents());
-        existingMissingComponents.removeAll(proposedFlow.getMissingComponents());
-
-        if (existingMissingComponents.size() > 0) {
-            final String missingIds = StringUtils.join(existingMissingComponents, ",");
-            return "Current flow has missing components that are not considered missing in the proposed flow (" + missingIds + ")";
-        }
-
-        final Set<String> proposedMissingComponents = new HashSet<>(proposedFlow.getMissingComponents());
-        proposedMissingComponents.removeAll(existingFlow.getMissingComponents());
-
-        if (proposedMissingComponents.size() > 0) {
-            final String missingIds = StringUtils.join(proposedMissingComponents, ",");
-            return "Proposed flow has missing components that are not considered missing in the current flow (" + missingIds + ")";
-        }
-
-        return null;
-    }
-
-    /**
-     * If both authorizers are external authorizers, or if the both are internal
-     * authorizers with equal fingerprints, then an uniheritable result with no
-     * reason is returned to indicate nothing to do.
-     *
-     * If both are internal authorizers and the current authorizer is empty,
-     * then an inheritable result is returned.
-     *
-     * All other cases return uninheritable with a reason which indicates to
-     * throw an exception.
-     *
-     * @param existingFlow the existing DataFlow
-     * @param proposedFlow the proposed DataFlow
-     * @return the AuthorizerInheritability result
-     */
-    private AuthorizerInheritability checkAuthorizerInheritability(final Authorizer authorizer, final DataFlow existingFlow, final DataFlow proposedFlow) {
-        final byte[] existing = existingFlow.getAuthorizerFingerprint();
-        final byte[] proposed = proposedFlow.getAuthorizerFingerprint();
-
-        // both are using external authorizers so nothing to inherit, but we don't want to throw an exception
-        if (existing == null && proposed == null) {
-            return AuthorizerInheritability.uninheritable(null);
-        }
-
-        // current is external, but proposed is internal
-        if (existing == null && proposed != null) {
-            return AuthorizerInheritability.uninheritable(
-                    "Current Authorizer is an external Authorizer, but proposed Authorizer is an internal Authorizer");
-        }
-
-        // current is internal, but proposed is external
-        if (existing != null && proposed == null) {
-            return AuthorizerInheritability.uninheritable(
-                    "Current Authorizer is an internal Authorizer, but proposed Authorizer is an external Authorizer");
-        }
-
-        // both are internal, but not the same
-        if (!Arrays.equals(existing, proposed)) {
-            if (AuthorizerCapabilityDetection.isManagedAuthorizer(authorizer)) {
-                final ManagedAuthorizer managedAuthorizer = (ManagedAuthorizer) authorizer;
-
-                try {
-                    // if the configurations are not equal, see if the manager indicates the proposed configuration is inheritable
-                    managedAuthorizer.checkInheritability(new String(proposed, StandardCharsets.UTF_8));
-                    return AuthorizerInheritability.inheritable();
-                } catch (final UninheritableAuthorizationsException e) {
-                    return AuthorizerInheritability.uninheritable("Proposed Authorizations do not match current Authorizations: " + e.getMessage());
-                }
-            } else {
-                // should never hit since the existing is only null when authorizer is not managed
-                return AuthorizerInheritability.uninheritable(
-                        "Proposed Authorizations do not match current Authorizations and are not configured with an internal Authorizer");
-            }
-        }
-
-        // both are internal and equal
-        return AuthorizerInheritability.uninheritable(null);
-    }
-
-    /**
-     * Returns true if the given controller can inherit the proposed flow
-     * without orphaning flow files.
-     *
-     * @param existingFlow flow
-     * @param controller the running controller
-     * @param proposedFlow the flow to inherit
-     *
-     * @return null if the controller can inherit the specified flow, an
-     * explanation of why it cannot be inherited otherwise
-     *
-     * @throws FingerprintException if flow fingerprints could not be generated
-     */
-    public String checkFlowInheritability(final DataFlow existingFlow, final DataFlow proposedFlow, final FlowController controller) throws FingerprintException {
-        if (existingFlow == null) {
-            return null;  // no existing flow, so equivalent to proposed flow
-        }
-
-        return checkFlowInheritability(existingFlow.getFlow(), proposedFlow.getFlow(), controller);
     }
 
-    private String checkFlowInheritability(final byte[] existingFlow, final byte[] proposedFlow, final FlowController controller) {
-        if (existingFlow == null) {
-            return null; // no existing flow, so equivalent to proposed flow
-        }
-
-        // check if the Flow is inheritable
-        final FingerprintFactory fingerprintFactory = new FingerprintFactory(encryptor, extensionManager);
-        final String existingFlowFingerprintBeforeHash = fingerprintFactory.createFingerprint(existingFlow, controller);
-        if (existingFlowFingerprintBeforeHash.trim().isEmpty()) {
-            return null;  // no existing flow, so equivalent to proposed flow
-        }
-
-        if (proposedFlow == null || proposedFlow.length == 0) {
-            return "Proposed Flow was empty but Current Flow is not";  // existing flow is not empty and proposed flow is empty (we could orphan flowfiles)
-        }
-
-        final String proposedFlowFingerprintBeforeHash = fingerprintFactory.createFingerprint(proposedFlow, controller);
-        if (proposedFlowFingerprintBeforeHash.trim().isEmpty()) {
-            return "Proposed Flow was empty but Current Flow is not";  // existing flow is not empty and proposed flow is empty (we could orphan flowfiles)
-        }
-
-        if (logger.isTraceEnabled()) {
-            logger.trace("Local Fingerprint Before Hash = {}", new Object[] {existingFlowFingerprintBeforeHash});
-            logger.trace("Proposed Fingerprint Before Hash = {}", new Object[] {proposedFlowFingerprintBeforeHash});
-        }
-
-        final boolean inheritable = existingFlowFingerprintBeforeHash.equals(proposedFlowFingerprintBeforeHash);
-        if (!inheritable) {
-            return findFirstDiscrepancy(existingFlowFingerprintBeforeHash, proposedFlowFingerprintBeforeHash, "Flows");
-        }
-
-        return null;
-    }
-
-    private String findFirstDiscrepancy(final String existing, final String proposed, final String comparisonDescription) {
-        final int shortestFileLength = Math.min(existing.length(), proposed.length());
-        for (int i = 0; i < shortestFileLength; i++) {
-            if (existing.charAt(i) != proposed.charAt(i)) {
-                final String formattedExistingDelta = formatFlowDiscrepancy(existing, i, 100);
-                final String formattedProposedDelta = formatFlowDiscrepancy(proposed, i, 100);
-                return String.format("Found difference in %s:\nLocal Fingerprint:   %s\nCluster Fingerprint: %s", comparisonDescription, formattedExistingDelta, formattedProposedDelta);
-            }
-        }
-
-        // existing must startWith proposed or proposed must startWith existing
-        if (existing.length() > proposed.length()) {
-            final String formattedExistingDelta = existing.substring(proposed.length(), Math.min(existing.length(), proposed.length() + 200));
-            return String.format("Found difference in %s:\nLocal Fingerprint contains additional configuration from Cluster Fingerprint: %s", comparisonDescription, formattedExistingDelta);
-        } else if (proposed.length() > existing.length()) {
-            final String formattedProposedDelta = proposed.substring(existing.length(), Math.min(proposed.length(), existing.length() + 200));
-            return String.format("Found difference in %s:\nCluster Fingerprint contains additional configuration from Local Fingerprint: %s", comparisonDescription, formattedProposedDelta);
-        }
-
-        return "Unable to find any discrepancies between fingerprints. Please contact the NiFi support team";
-    }
 
     private byte[] toBytes(final FlowController flowController) throws FlowSerializationException {
         final ByteArrayOutputStream result = new ByteArrayOutputStream();
@@ -1839,41 +1717,4 @@ public class StandardFlowSynchronizer implements FlowSynchronizer {
 
         return matches;
     }
-
-    private String formatFlowDiscrepancy(final String flowFingerprint, final int deltaIndex, final int deltaPad) {
-        return flowFingerprint.substring(Math.max(0, deltaIndex - deltaPad), Math.min(flowFingerprint.length(), deltaIndex + deltaPad));
-    }
-
-    /**
-     * Holder for the result of determining if a proposed Authorizer is
-     * inheritable.
-     */
-    private static final class AuthorizerInheritability {
-
-        private final boolean inheritable;
-        private final String reason;
-
-        public AuthorizerInheritability(boolean inheritable, String reason) {
-            this.inheritable = inheritable;
-            this.reason = reason;
-        }
-
-        public boolean isInheritable() {
-            return inheritable;
-        }
-
-        public String getReason() {
-            return reason;
-        }
-
-        public static AuthorizerInheritability uninheritable(String reason) {
-            return new AuthorizerInheritability(false, reason);
-        }
-
-        public static AuthorizerInheritability inheritable() {
-            return new AuthorizerInheritability(true, null);
-        }
-
-    }
-
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/flow/StandardFlowManager.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/flow/StandardFlowManager.java
index fd74d54..4663d62 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/flow/StandardFlowManager.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/flow/StandardFlowManager.java
@@ -68,6 +68,7 @@ import org.apache.nifi.parameter.ParameterReferenceManager;
 import org.apache.nifi.parameter.StandardParameterContext;
 import org.apache.nifi.parameter.StandardParameterReferenceManager;
 import org.apache.nifi.registry.VariableRegistry;
+import org.apache.nifi.registry.flow.FlowRegistryClient;
 import org.apache.nifi.registry.variable.MutableVariableRegistry;
 import org.apache.nifi.remote.PublicPort;
 import org.apache.nifi.remote.RemoteGroupPort;
@@ -753,4 +754,55 @@ public class StandardFlowManager implements FlowManager {
         return parameterContext;
     }
 
+    @Override
+    public void purge() {
+        verifyCanPurge();
+
+        final ProcessGroup rootGroup = getRootGroup();
+
+        // Delete templates from all levels first. This allows us to avoid having to purge each individual Process Group recursively
+        // and instead just delete all child Process Groups after removing the connections to/from those Process Groups.
+        for (final ProcessGroup group : rootGroup.findAllProcessGroups()) {
+            group.getTemplates().forEach(group::removeTemplate);
+        }
+        rootGroup.getTemplates().forEach(rootGroup::removeTemplate);
+
+        rootGroup.getConnections().forEach(rootGroup::removeConnection);
+        rootGroup.getProcessors().forEach(rootGroup::removeProcessor);
+        rootGroup.getFunnels().forEach(rootGroup::removeFunnel);
+        rootGroup.getInputPorts().forEach(rootGroup::removeInputPort);
+        rootGroup.getOutputPorts().forEach(rootGroup::removeOutputPort);
+        rootGroup.getLabels().forEach(rootGroup::removeLabel);
+        rootGroup.getRemoteProcessGroups().forEach(rootGroup::removeRemoteProcessGroup);
+
+        rootGroup.getProcessGroups().forEach(rootGroup::removeProcessGroup);
+
+        final ControllerServiceProvider serviceProvider = flowController.getControllerServiceProvider();
+        rootGroup.getControllerServices(false).forEach(serviceProvider::removeControllerService);
+
+        getRootControllerServices().forEach(this::removeRootControllerService);
+        getAllReportingTasks().forEach(this::removeReportingTask);
+
+        final FlowRegistryClient registryClient = flowController.getFlowRegistryClient();
+        for (final String registryId : registryClient.getRegistryIdentifiers()) {
+            registryClient.removeFlowRegistry(registryId);
+        }
+
+        for (final ParameterContext parameterContext : parameterContextManager.getParameterContexts()) {
+            parameterContextManager.removeParameterContext(parameterContext.getIdentifier());
+        }
+    }
+
+    private void verifyCanPurge() {
+        for (final ControllerServiceNode serviceNode : getAllControllerServices()) {
+            serviceNode.verifyCanDelete();
+        }
+
+        for (final ReportingTaskNode reportingTask : getAllReportingTasks()) {
+            reportingTask.verifyCanDelete();
+        }
+
+        final ProcessGroup rootGroup = getRootGroup();
+        rootGroup.verifyCanDelete(true, true);
+    }
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/inheritance/AuthorizerCheck.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/inheritance/AuthorizerCheck.java
new file mode 100644
index 0000000..3217bef
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/inheritance/AuthorizerCheck.java
@@ -0,0 +1,75 @@
+/*
+ * 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.inheritance;
+
+import org.apache.nifi.authorization.Authorizer;
+import org.apache.nifi.authorization.AuthorizerCapabilityDetection;
+import org.apache.nifi.authorization.ManagedAuthorizer;
+import org.apache.nifi.authorization.exception.UninheritableAuthorizationsException;
+import org.apache.nifi.cluster.protocol.DataFlow;
+import org.apache.nifi.controller.FlowController;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+
+public class AuthorizerCheck implements FlowInheritabilityCheck {
+    @Override
+    public FlowInheritability checkInheritability(final DataFlow existingFlow, final DataFlow proposedFlow, final FlowController flowController) {
+        final byte[] existing = existingFlow.getAuthorizerFingerprint();
+        final byte[] proposed = proposedFlow.getAuthorizerFingerprint();
+
+        // both are using external authorizers so nothing to inherit, but we don't want to throw an exception
+        if (existing == null && proposed == null) {
+            return FlowInheritability.notInheritable(null);
+        }
+
+        // current is external, but proposed is internal
+        if (existing == null && proposed != null) {
+            return FlowInheritability.notInheritable(
+                "Current Authorizer is an external Authorizer, but proposed Authorizer is an internal Authorizer");
+        }
+
+        // current is internal, but proposed is external
+        if (existing != null && proposed == null) {
+            return FlowInheritability.notInheritable(
+                "Current Authorizer is an internal Authorizer, but proposed Authorizer is an external Authorizer");
+        }
+
+        // both are internal, but not the same
+        if (!Arrays.equals(existing, proposed)) {
+            final Authorizer authorizer = flowController.getAuthorizer();
+
+            if (AuthorizerCapabilityDetection.isManagedAuthorizer(authorizer)) {
+                final ManagedAuthorizer managedAuthorizer = (ManagedAuthorizer) authorizer;
+
+                try {
+                    // if the configurations are not equal, see if the manager indicates the proposed configuration is inheritable
+                    managedAuthorizer.checkInheritability(new String(proposed, StandardCharsets.UTF_8));
+                    return FlowInheritability.inheritable();
+                } catch (final UninheritableAuthorizationsException e) {
+                    return FlowInheritability.notInheritable("Proposed Authorizations do not match current Authorizations: " + e.getMessage());
+                }
+            } else {
+                // should never hit since the existing is only null when authorizer is not managed
+                return FlowInheritability.notInheritable("Proposed Authorizations do not match current Authorizations and are not configured with an internal Authorizer");
+            }
+        }
+
+        // both are internal and equal
+        return FlowInheritability.notInheritable(null);
+    }
+}
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
new file mode 100644
index 0000000..504f582
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/inheritance/BundleCompatibilityCheck.java
@@ -0,0 +1,81 @@
+/*
+ * 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.inheritance;
+
+import org.apache.nifi.cluster.protocol.DataFlow;
+import org.apache.nifi.controller.FlowController;
+import org.apache.nifi.controller.serialization.FlowFromDOMFactory;
+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.w3c.dom.Document;
+import org.w3c.dom.Element;
+import org.w3c.dom.Node;
+import org.w3c.dom.NodeList;
+
+public class BundleCompatibilityCheck implements FlowInheritabilityCheck {
+
+    @Override
+    public FlowInheritability checkInheritability(final DataFlow existingFlow, final DataFlow proposedFlow, final FlowController flowController) {
+        final Document configuration = proposedFlow.getFlowDocument();
+
+        if (configuration == null) {
+            return FlowInheritability.inheritable();
+        }
+
+        final ExtensionManager extensionManager = flowController.getExtensionManager();
+        final NodeList bundleNodes = configuration.getElementsByTagName("bundle");
+        for (int i = 0; i < bundleNodes.getLength(); i++) {
+            final Node bundleNode = bundleNodes.item(i);
+            if (bundleNode instanceof Element) {
+                final Element bundleElement = (Element) bundleNode;
+
+                final Node componentNode = bundleElement.getParentNode();
+                if (componentNode instanceof Element) {
+                    final Element componentElement = (Element) componentNode;
+                    if (!withinTemplate(componentElement)) {
+                        final String componentType = DomUtils.getChildText(componentElement, "class");
+                        final BundleDTO bundleDto = FlowFromDOMFactory.getBundle(bundleElement);
+
+                        try {
+                            BundleUtils.getBundle(extensionManager, componentType, bundleDto);
+                        } catch (final IllegalStateException e) {
+                            final String bundleDescription = bundleDto.getGroup() + ":" + bundleDto.getArtifact() + ":" + bundleDto.getVersion();
+                            return FlowInheritability.notInheritable("Could not find Bundle " + bundleDescription + ": " + e.getMessage());
+                        }
+                    }
+                }
+            }
+        }
+
+        return FlowInheritability.inheritable();
+    }
+
+    private boolean withinTemplate(final Element element) {
+        if ("template".equals(element.getTagName())) {
+            return true;
+        } else {
+            final Node parentNode = element.getParentNode();
+            if (parentNode instanceof Element) {
+                return withinTemplate((Element) parentNode);
+            } else {
+                return false;
+            }
+        }
+    }
+}
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
new file mode 100644
index 0000000..34fa87a
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/inheritance/ConnectionMissingCheck.java
@@ -0,0 +1,87 @@
+/*
+ * 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.inheritance;
+
+import org.apache.nifi.cluster.protocol.DataFlow;
+import org.apache.nifi.controller.FlowController;
+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.web.api.dto.ConnectionDTO;
+import org.apache.nifi.web.api.dto.ProcessGroupDTO;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.w3c.dom.Document;
+import org.w3c.dom.Element;
+
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.Set;
+
+/**
+ * Determines whether or not the proposed flow can be inherited based on whether or not it has all of the Connections that locally have data queued.
+ * If the local flow has any connection in which data is queued, that connection must exist in the proposed flow, or else the flow will be considered uninheritable.
+ */
+public class ConnectionMissingCheck implements FlowInheritabilityCheck {
+    private static final Logger logger = LoggerFactory.getLogger(ConnectionMissingCheck.class);
+
+    @Override
+    public FlowInheritability checkInheritability(final DataFlow existingFlow, final DataFlow proposedFlow, final FlowController flowController) {
+        final Document flowDocument = proposedFlow.getFlowDocument();
+        final Element rootGroupElement = (Element) flowDocument.getDocumentElement().getElementsByTagName("rootGroup").item(0);
+        final FlowEncodingVersion encodingVersion = FlowEncodingVersion.parse(flowDocument.getDocumentElement());
+
+        final ProcessGroupDTO rootGroupDto = FlowFromDOMFactory.getProcessGroup(null, rootGroupElement, null, encodingVersion);
+        final Set<String> connectionIds = findAllConnectionIds(rootGroupDto);
+
+        final FlowFileRepository flowFileRepository = flowController.getRepositoryContextFactory().getFlowFileRepository();
+
+        final Set<String> queuesWithFlowFiles;
+        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 String queueId : queuesWithFlowFiles) {
+            if (!connectionIds.contains(queueId)) {
+                return FlowInheritability.notInheritable("Proposed Flow does not contain a Connection with ID " + queueId + " but this instance has data queued in that connection");
+            }
+        }
+
+        return FlowInheritability.inheritable();
+    }
+
+    private Set<String> findAllConnectionIds(final ProcessGroupDTO group) {
+        final Set<String> connectionIds = new HashSet<>();
+        findAllConnectionIds(group, connectionIds);
+        return connectionIds;
+    }
+
+    private void findAllConnectionIds(final ProcessGroupDTO group, final Set<String> ids) {
+        for (final ConnectionDTO connectionDTO : group.getContents().getConnections()) {
+            ids.add(connectionDTO.getId());
+        }
+
+        for (final ProcessGroupDTO childGroup : group.getContents().getProcessGroups()) {
+            findAllConnectionIds(childGroup, ids);
+        }
+    }
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/inheritance/FlowFingerprintCheck.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/inheritance/FlowFingerprintCheck.java
new file mode 100644
index 0000000..9661847
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/inheritance/FlowFingerprintCheck.java
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.controller.inheritance;
+
+import org.apache.nifi.cluster.protocol.DataFlow;
+import org.apache.nifi.controller.FlowController;
+import org.apache.nifi.encrypt.StringEncryptor;
+import org.apache.nifi.fingerprint.FingerprintFactory;
+import org.apache.nifi.nar.ExtensionManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class FlowFingerprintCheck implements FlowInheritabilityCheck {
+    private static final Logger logger = LoggerFactory.getLogger(FlowFingerprintCheck.class);
+
+    @Override
+    public FlowInheritability checkInheritability(final DataFlow existingFlow, final DataFlow proposedFlow, final FlowController flowController) {
+        if (existingFlow == null) {
+            return FlowInheritability.inheritable();
+        }
+
+        final byte[] existingFlowBytes = existingFlow.getFlow();
+        final byte[] proposedFlowBytes = proposedFlow.getFlow();
+
+        final StringEncryptor encryptor = flowController.getEncryptor();
+        final ExtensionManager extensionManager = flowController.getExtensionManager();
+
+        final FingerprintFactory fingerprintFactory = new FingerprintFactory(encryptor, extensionManager);
+        final String existingFlowFingerprintBeforeHash = fingerprintFactory.createFingerprint(existingFlowBytes, flowController);
+        if (existingFlowFingerprintBeforeHash.trim().isEmpty()) {
+            return null;  // no existing flow, so equivalent to proposed flow
+        }
+
+        if (proposedFlow == null || proposedFlowBytes.length == 0) {
+            return FlowInheritability.notInheritable("Proposed Flow was empty but Current Flow is not");  // existing flow is not empty and proposed flow is empty (we could orphan flowfiles)
+        }
+
+        final String proposedFlowFingerprintBeforeHash = fingerprintFactory.createFingerprint(proposedFlow.getFlowDocument(), flowController);
+        if (proposedFlowFingerprintBeforeHash.trim().isEmpty()) {
+            return FlowInheritability.notInheritable("Proposed Flow was empty but Current Flow is not");  // existing flow is not empty and proposed flow is empty (we could orphan flowfiles)
+        }
+
+        if (logger.isTraceEnabled()) {
+            logger.trace("Local Fingerprint Before Hash = {}", new Object[] {existingFlowFingerprintBeforeHash});
+            logger.trace("Proposed Fingerprint Before Hash = {}", new Object[] {proposedFlowFingerprintBeforeHash});
+        }
+
+        final boolean inheritable = existingFlowFingerprintBeforeHash.equals(proposedFlowFingerprintBeforeHash);
+        if (!inheritable) {
+            final String discrepancy = findFirstDiscrepancy(existingFlowFingerprintBeforeHash, proposedFlowFingerprintBeforeHash, "Flows");
+            return FlowInheritability.notInheritable(discrepancy);
+        }
+
+        return FlowInheritability.inheritable();
+    }
+
+    private String findFirstDiscrepancy(final String existing, final String proposed, final String comparisonDescription) {
+        final int shortestFileLength = Math.min(existing.length(), proposed.length());
+        for (int i = 0; i < shortestFileLength; i++) {
+            if (existing.charAt(i) != proposed.charAt(i)) {
+                final String formattedExistingDelta = formatFlowDiscrepancy(existing, i, 100);
+                final String formattedProposedDelta = formatFlowDiscrepancy(proposed, i, 100);
+                return String.format("Found difference in %s:\nLocal Fingerprint:   %s\nCluster Fingerprint: %s", comparisonDescription, formattedExistingDelta, formattedProposedDelta);
+            }
+        }
+
+        // existing must startWith proposed or proposed must startWith existing
+        if (existing.length() > proposed.length()) {
+            final String formattedExistingDelta = existing.substring(proposed.length(), Math.min(existing.length(), proposed.length() + 200));
+            return String.format("Found difference in %s:\nLocal Fingerprint contains additional configuration from Cluster Fingerprint: %s", comparisonDescription, formattedExistingDelta);
+        } else if (proposed.length() > existing.length()) {
+            final String formattedProposedDelta = proposed.substring(existing.length(), Math.min(proposed.length(), existing.length() + 200));
+            return String.format("Found difference in %s:\nCluster Fingerprint contains additional configuration from Local Fingerprint: %s", comparisonDescription, formattedProposedDelta);
+        }
+
+        return "Unable to find any discrepancies between fingerprints. Please contact the NiFi support team";
+    }
+
+    private String formatFlowDiscrepancy(final String flowFingerprint, final int deltaIndex, final int deltaPad) {
+        return flowFingerprint.substring(Math.max(0, deltaIndex - deltaPad), Math.min(flowFingerprint.length(), deltaIndex + deltaPad));
+    }
+
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/inheritance/FlowInheritability.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/inheritance/FlowInheritability.java
new file mode 100644
index 0000000..9ce18a6
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/inheritance/FlowInheritability.java
@@ -0,0 +1,68 @@
+/*
+ * 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.inheritance;
+
+public interface FlowInheritability {
+    /**
+     * @return whether or not the flow can be inherited
+     */
+    boolean isInheritable();
+
+    /**
+     * If the flow is not inheritable, this provides an explanation as to why the flow cannot be inherited
+     * @return an explanation as to why the flow cannot be inherited, or <code>null</code> if the flow is inheritable
+     */
+    String getExplanation();
+
+    static FlowInheritability inheritable() {
+        return new FlowInheritability() {
+            @Override
+            public boolean isInheritable() {
+                return true;
+            }
+
+            @Override
+            public String getExplanation() {
+                return null;
+            }
+
+            @Override
+            public String toString() {
+                return "FlowInheritability[inheritable=true]";
+            }
+        };
+    }
+
+    static FlowInheritability notInheritable(String explanation) {
+        return new FlowInheritability() {
+            @Override
+            public boolean isInheritable() {
+                return false;
+            }
+
+            @Override
+            public String getExplanation() {
+                return explanation;
+            }
+
+            @Override
+            public String toString() {
+                return "FlowInheritability[inheritable=false, explanation=" + explanation + "]";
+            }
+        };
+    }
+}
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/src/main/java/org/apache/nifi/controller/inheritance/FlowInheritabilityCheck.java
similarity index 51%
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/src/main/java/org/apache/nifi/controller/inheritance/FlowInheritabilityCheck.java
index 0317584..cded2fb 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/src/main/java/org/apache/nifi/controller/inheritance/FlowInheritabilityCheck.java
@@ -14,31 +14,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.nifi.cluster.protocol;
+package org.apache.nifi.controller.inheritance;
 
-import java.util.Set;
-
-public interface DataFlow {
-
-    /**
-     * @return the raw byte array of the flow
-     */
-    public byte[] getFlow();
+import org.apache.nifi.cluster.protocol.DataFlow;
+import org.apache.nifi.controller.FlowController;
 
+public interface FlowInheritabilityCheck {
     /**
-     * @return the raw byte array of the snippets
+     * Determines whether or not the given proposed flow can be inherited, based on the given existing flow
+     *
+     * @param existingFlow the existing flow
+     * @param proposedFlow the flow that is being proposed for inheritance
+     * @param flowController the FlowController that can be used to understand additional context about the current state
+     *
+     * @return a FlowInheritability indicating whether or not the flow is inheritable
      */
-    public byte[] getSnippets();
-
-    /**
-     * @return the raw byte array of the Authorizer's fingerprint,
-     *              null when not using a ManagedAuthorizer
-     */
-    public byte[] getAuthorizerFingerprint();
-
-    /**
-     * @return the component ids of components that were created as a missing ghost component
-     */
-    public Set<String> getMissingComponents();
-
+    FlowInheritability checkInheritability(DataFlow existingFlow, DataFlow proposedFlow, FlowController flowController);
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/inheritance/MissingComponentsCheck.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/inheritance/MissingComponentsCheck.java
new file mode 100644
index 0000000..fd7984b
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/inheritance/MissingComponentsCheck.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.controller.inheritance;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.cluster.protocol.DataFlow;
+import org.apache.nifi.controller.FlowController;
+
+import java.util.HashSet;
+import java.util.Set;
+
+public class MissingComponentsCheck implements FlowInheritabilityCheck {
+    @Override
+    public FlowInheritability checkInheritability(final DataFlow existingFlow, final DataFlow proposedFlow, final FlowController flowController) {
+        if (existingFlow == null) {
+            return FlowInheritability.inheritable(); // no existing flow, so equivalent to proposed flow
+        }
+
+        final Set<String> existingMissingComponents = new HashSet<>(existingFlow.getMissingComponents());
+        existingMissingComponents.removeAll(proposedFlow.getMissingComponents());
+
+        if (existingMissingComponents.size() > 0) {
+            final String missingIds = StringUtils.join(existingMissingComponents, ",");
+            return FlowInheritability.notInheritable("Current flow has missing components that are not considered missing in the proposed flow (" + missingIds + ")");
+        }
+
+        final Set<String> proposedMissingComponents = new HashSet<>(proposedFlow.getMissingComponents());
+        proposedMissingComponents.removeAll(existingFlow.getMissingComponents());
+
+        if (proposedMissingComponents.size() > 0) {
+            final String missingIds = StringUtils.join(proposedMissingComponents, ",");
+            return FlowInheritability.notInheritable("Proposed flow has missing components that are not considered missing in the current flow (" + missingIds + ")");
+        }
+
+        return FlowInheritability.inheritable();
+    }
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/LiveSerializedRepositoryRecord.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/LiveSerializedRepositoryRecord.java
new file mode 100644
index 0000000..605ae46
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/LiveSerializedRepositoryRecord.java
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.controller.repository;
+
+import org.apache.nifi.controller.queue.FlowFileQueue;
+import org.apache.nifi.controller.repository.claim.ContentClaim;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+
+public class LiveSerializedRepositoryRecord implements SerializedRepositoryRecord {
+    private final RepositoryRecord record;
+
+    public LiveSerializedRepositoryRecord(final RepositoryRecord repositoryRecord) {
+        this.record = repositoryRecord;
+    }
+
+    @Override
+    public String getQueueIdentifier() {
+        final FlowFileQueue destination = record.getDestination();
+        final FlowFileQueue queue = destination == null ? record.getOriginalQueue() : destination;
+        return queue == null ? null : queue.getIdentifier();
+    }
+
+    @Override
+    public RepositoryRecordType getType() {
+        return record.getType();
+    }
+
+    @Override
+    public ContentClaim getContentClaim() {
+        return record.getCurrentClaim();
+    }
+
+    @Override
+    public long getClaimOffset() {
+        return record.getCurrentClaimOffset();
+    }
+
+    @Override
+    public String getSwapLocation() {
+        return record.getSwapLocation();
+    }
+
+    @Override
+    public FlowFileRecord getFlowFileRecord() {
+        return record.getCurrent();
+    }
+
+    @Override
+    public boolean isMarkedForAbort() {
+        return record.isMarkedForAbort();
+    }
+
+    @Override
+    public boolean isAttributesChanged() {
+        return record.isAttributesChanged();
+    }
+
+    @Override
+    public String toString() {
+        return "LiveSerializedRepositoryRecord[recordType=" + record.getType() + ", queueId=" + (record.getDestination() == null ? null : record.getDestination().getIdentifier())
+            + ", flowFileUuid=" + record.getCurrent().getAttribute(CoreAttributes.UUID.key()) + ", attributesChanged=" + isAttributesChanged() + "]";
+    }
+
+
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/RocksDBFlowFileRepository.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/RocksDBFlowFileRepository.java
index 4f3dd45..105967b 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/RocksDBFlowFileRepository.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/RocksDBFlowFileRepository.java
@@ -91,6 +91,8 @@ public class RocksDBFlowFileRepository implements FlowFileRepository {
     private static final Runtime runtime = Runtime.getRuntime();
     private static final NumberFormat percentFormat = NumberFormat.getPercentInstance();
 
+    private final Map<String, FlowFileQueue> queueMap = new HashMap<>();
+
     /**
      * Each property is defined by its name in the file and its default value
      */
@@ -238,7 +240,7 @@ public class RocksDBFlowFileRepository implements FlowFileRepository {
     private final RocksDBMetronome db;
     private ResourceClaimManager claimManager;
     private RepositoryRecordSerdeFactory serdeFactory;
-    private SerDe<RepositoryRecord> serializer;
+    private SerDe<SerializedRepositoryRecord> serializer;
     private String serializationEncodingName;
     private byte[] serializationHeader;
 
@@ -247,7 +249,7 @@ public class RocksDBFlowFileRepository implements FlowFileRepository {
     private final boolean removeOrphanedFlowFiles;
     private final boolean enableRecoveryMode;
     private final long recoveryModeFlowFileLimit;
-    private final AtomicReference<SerDe<RepositoryRecord>> recordDeserializer = new AtomicReference<>();
+    private final AtomicReference<SerDe<SerializedRepositoryRecord>> recordDeserializer = new AtomicReference<>();
     private final List<byte[]> recordsToRestore = Collections.synchronizedList(new LinkedList<>());
 
     private final ReentrantLock stallStopLock = new ReentrantLock();
@@ -550,7 +552,7 @@ public class RocksDBFlowFileRepository implements FlowFileRepository {
         // if we are not in recovery mode, return
         if (!enableRecoveryMode) return;
 
-        SerDe<RepositoryRecord> deserializer = recordDeserializer.get();
+        SerDe<SerializedRepositoryRecord> deserializer = recordDeserializer.get();
         if (deserializer == null) {
             return; // initial load hasn't completed
         }
@@ -577,9 +579,10 @@ public class RocksDBFlowFileRepository implements FlowFileRepository {
                 if (recordBytes != null) {
                     try (ByteArrayInputStream byteArrayInputStream = new ByteArrayInputStream(recordBytes);
                          DataInputStream dataInputStream = new DataInputStream(byteArrayInputStream)) {
-                        RepositoryRecord record = deserializer.deserializeRecord(dataInputStream, deserializer.getVersion());
-                        final FlowFileRecord flowFile = record.getCurrent();
-                        final FlowFileQueue queue = record.getOriginalQueue();
+                        SerializedRepositoryRecord record = deserializer.deserializeRecord(dataInputStream, deserializer.getVersion());
+                        final FlowFileRecord flowFile = record.getFlowFileRecord();
+
+                        final FlowFileQueue queue = queueMap.get(record.getQueueIdentifier());
                         if (queue != null) {
                             queue.put(flowFile);
                             inMemoryFlowFiles.incrementAndGet();
@@ -602,7 +605,7 @@ public class RocksDBFlowFileRepository implements FlowFileRepository {
     }
 
     /**
-     * Updates the FlowFile repository with the given RepositoryRecords
+     * Updates the FlowFile repository with the given SerializedRepositoryRecords
      *
      * @param records the records to update the repository with
      * @throws IOException if update fails or a required sync is interrupted
@@ -694,7 +697,7 @@ public class RocksDBFlowFileRepository implements FlowFileRepository {
             if (repositoryRecord.getType() == RepositoryRecordType.CLEANUP_TRANSIENT_CLAIMS) {
                 continue;
             }
-            final UpdateType updateType = serdeFactory.getUpdateType(repositoryRecord);
+            final UpdateType updateType = serdeFactory.getUpdateType(new LiveSerializedRepositoryRecord(repositoryRecord));
             partitionedRecords.computeIfAbsent(updateType, ut -> new ArrayList<>()).add(repositoryRecord);
         }
 
@@ -708,14 +711,16 @@ public class RocksDBFlowFileRepository implements FlowFileRepository {
             List<RepositoryRecord> swapOutRecords = partitionedRecords.get(UpdateType.SWAP_OUT);
             if (swapOutRecords != null) {
                 for (final RepositoryRecord record : swapOutRecords) {
-                    final String newLocation = serdeFactory.getLocation(record);
+                    final SerializedRepositoryRecord serializedRecord = new LiveSerializedRepositoryRecord(record);
+                    final String newLocation = serdeFactory.getLocation(serializedRecord);
+                    final Long recordIdentifier = serdeFactory.getRecordIdentifier(serializedRecord);
+
                     if (newLocation == null) {
-                        final Long recordIdentifier = serdeFactory.getRecordIdentifier(record);
                         logger.error("Received Record (ID=" + recordIdentifier + ") with UpdateType of SWAP_OUT but " +
                                 "no indicator of where the Record is to be Swapped Out to; these records may be " +
                                 "lost when the repository is restored!");
                     } else {
-                        delete(record);
+                        delete(recordIdentifier);
                     }
                 }
             }
@@ -724,9 +729,11 @@ public class RocksDBFlowFileRepository implements FlowFileRepository {
             List<RepositoryRecord> swapInRecords = partitionedRecords.get(UpdateType.SWAP_IN);
             if (swapInRecords != null) {
                 for (final RepositoryRecord record : swapInRecords) {
-                    final String newLocation = serdeFactory.getLocation(record);
+                    final SerializedRepositoryRecord serialized = new LiveSerializedRepositoryRecord(record);
+
+                    final String newLocation = serdeFactory.getLocation(serialized);
                     if (newLocation == null) {
-                        final Long recordIdentifier = serdeFactory.getRecordIdentifier(record);
+                        final Long recordIdentifier = serdeFactory.getRecordIdentifier(serialized);
                         logger.error("Received Record (ID=" + recordIdentifier + ") with UpdateType of SWAP_IN but " +
                                 "no indicator of where the Record is to be Swapped In from; these records may be " +
                                 "duplicated when the repository is restored!");
@@ -763,14 +770,15 @@ public class RocksDBFlowFileRepository implements FlowFileRepository {
     private void deleteAll(List<RepositoryRecord> repositoryRecords) throws RocksDBException {
         if (repositoryRecords != null) {
             for (final RepositoryRecord record : repositoryRecords) {
-                delete(record);
+                final SerializedRepositoryRecord serialized = new LiveSerializedRepositoryRecord(record);
+                final Long id = serdeFactory.getRecordIdentifier(serialized);
+                delete(id);
             }
         }
     }
 
-    private void delete(RepositoryRecord record) throws RocksDBException {
-        final Long recordIdentifier = serdeFactory.getRecordIdentifier(record);
-        byte[] key = RocksDBMetronome.getBytes(recordIdentifier);
+    private void delete(Long recordId) throws RocksDBException {
+        byte[] key = RocksDBMetronome.getBytes(recordId);
         db.delete(key);
     }
 
@@ -783,7 +791,7 @@ public class RocksDBFlowFileRepository implements FlowFileRepository {
     }
 
     private void put(RepositoryRecord record) throws IOException, RocksDBException {
-        final Long recordIdentifier = serdeFactory.getRecordIdentifier(record);
+        final Long recordIdentifier = serdeFactory.getRecordIdentifier(new LiveSerializedRepositoryRecord(record));
         byte[] key = RocksDBMetronome.getBytes(recordIdentifier);
         final byte[] serializedRecord = serialize(record);
         db.put(key, serializedRecord);
@@ -792,7 +800,7 @@ public class RocksDBFlowFileRepository implements FlowFileRepository {
     private byte[] serialize(RepositoryRecord record) throws IOException {
         try (ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
              DataOutputStream dataOutputStream = new DataOutputStream(byteArrayOutputStream)) {
-            serializer.serializeRecord(record, dataOutputStream);
+            serializer.serializeRecord(new LiveSerializedRepositoryRecord(record), dataOutputStream);
             return byteArrayOutputStream.toByteArray();
         }
     }
@@ -997,7 +1005,7 @@ public class RocksDBFlowFileRepository implements FlowFileRepository {
 
         final long startTime = System.nanoTime();
 
-        final Map<String, FlowFileQueue> queueMap = new HashMap<>();
+        queueMap.clear();
         for (final FlowFileQueue queue : queueProvider.getAllQueues()) {
             queueMap.put(queue.getIdentifier(), queue);
         }
@@ -1015,7 +1023,6 @@ public class RocksDBFlowFileRepository implements FlowFileRepository {
         final List<Future<Long>> futures = new ArrayList<>(deserializationThreads);
 
         RepositoryRecordSerdeFactory factory = new StandardRepositoryRecordSerdeFactory(claimManager);
-        factory.setQueueMap(queueMap);
 
         final AtomicInteger numFlowFilesMissingQueue = new AtomicInteger(0);
         final AtomicInteger recordCount = new AtomicInteger(0);
@@ -1028,8 +1035,7 @@ public class RocksDBFlowFileRepository implements FlowFileRepository {
                 final Set<String> localRecoveredSwapLocations = new HashSet<>();
 
                 // Create deserializer in each thread
-                factory.setQueueMap(queueMap);
-                final SerDe<RepositoryRecord> localDeserializer = factory.createSerDe(serializationEncodingName);
+                final SerDe<SerializedRepositoryRecord> localDeserializer = factory.createSerDe(serializationEncodingName);
                 try (ByteArrayInputStream byteArrayInputStream = new ByteArrayInputStream(serializationHeader);
                      DataInputStream dataInputStream = new DataInputStream(byteArrayInputStream)) {
                     localDeserializer.readHeader(dataInputStream);
@@ -1040,17 +1046,17 @@ public class RocksDBFlowFileRepository implements FlowFileRepository {
                     if (value != null) {
                         try (ByteArrayInputStream byteArrayInputStream = new ByteArrayInputStream(value);
                              DataInputStream dataInputStream = new DataInputStream(byteArrayInputStream)) {
-                            RepositoryRecord record = localDeserializer.deserializeRecord(dataInputStream, localDeserializer.getVersion());
+                            SerializedRepositoryRecord record = localDeserializer.deserializeRecord(dataInputStream, localDeserializer.getVersion());
 
                             localRecordCount++;
 
                             // increment the count for the record
-                            final ContentClaim claim = record.getCurrentClaim();
+                            final ContentClaim claim = record.getContentClaim();
                             if (claim != null) {
                                 claimManager.incrementClaimantCount(claim.getResourceClaim());
                             }
 
-                            final long recordId = record.getCurrent().getId();
+                            final long recordId = record.getFlowFileRecord().getId();
                             if (recordId > localMaxId) {
                                 localMaxId = recordId;
                             }
@@ -1059,8 +1065,8 @@ public class RocksDBFlowFileRepository implements FlowFileRepository {
                                 localRecoveredSwapLocations.add(normalizeSwapLocation(record.getSwapLocation()));
                             }
 
-                            final FlowFileRecord flowFile = record.getCurrent();
-                            final FlowFileQueue queue = record.getOriginalQueue();
+                            final FlowFileRecord flowFile = record.getFlowFileRecord();
+                            final FlowFileQueue queue = queueMap.get(record.getQueueIdentifier());
                             if (queue == null) {
                                 if (!removeOrphanedFlowFiles) {
                                     throw new IOException("Found FlowFile in repository without a corresponding queue.  " +
@@ -1170,8 +1176,7 @@ public class RocksDBFlowFileRepository implements FlowFileRepository {
             logger.warn("On recovery, found {} FlowFiles whose queue no longer exists.  These FlowFiles have been dropped.", numFlowFilesMissingQueue);
         }
 
-        final SerDe<RepositoryRecord> deserializer = factory.createSerDe(serializationEncodingName);
-        factory.setQueueMap(null); // clear the map
+        final SerDe<SerializedRepositoryRecord> deserializer = factory.createSerDe(serializationEncodingName);
 
         try (ByteArrayInputStream byteArrayInputStream = new ByteArrayInputStream(serializationHeader);
              DataInputStream dataInputStream = new DataInputStream(byteArrayInputStream)) {
@@ -1185,6 +1190,10 @@ public class RocksDBFlowFileRepository implements FlowFileRepository {
         return maxId;
     }
 
+    @Override
+    public Set<String> findQueuesWithFlowFiles(final FlowFileSwapManager flowFileSwapManager) throws IOException {
+        return null;
+    }
 
     private void addRawSwapLocation(String rawSwapLocation) throws IOException {
         addRawSwapLocations(Collections.singleton(rawSwapLocation));
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/VolatileFlowFileRepository.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/VolatileFlowFileRepository.java
index 3881f09..7268947 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/VolatileFlowFileRepository.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/VolatileFlowFileRepository.java
@@ -23,8 +23,10 @@ import org.apache.nifi.controller.repository.claim.ResourceClaimManager;
 
 import java.io.IOException;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.List;
 import java.util.Objects;
+import java.util.Set;
 import java.util.concurrent.atomic.AtomicLong;
 
 /**
@@ -145,6 +147,11 @@ public class VolatileFlowFileRepository implements FlowFileRepository {
     }
 
     @Override
+    public Set<String> findQueuesWithFlowFiles(FlowFileSwapManager swapManager) throws IOException {
+        return Collections.emptySet();
+    }
+
+    @Override
     public void updateMaxFlowFileIdentifier(final long maxId) {
         while (true) {
             final long currentId = idGenerator.get();
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 d81ead3..4b095ed 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
@@ -105,12 +105,14 @@ public class WriteAheadFlowFileRepository implements FlowFileRepository, SyncLis
     private final int numPartitions;
     final ScheduledExecutorService checkpointExecutor;
 
-    final Set<String> swapLocationSuffixes = new HashSet<>(); // guarded by synchronizing on object itself
+    private volatile Collection<SerializedRepositoryRecord> recoveredRecords = null;
+
+    private final Set<String> swapLocationSuffixes = new HashSet<>(); // guarded by synchronizing on object itself
 
     // effectively final
-    private WriteAheadRepository<RepositoryRecord> wal;
-    RepositoryRecordSerdeFactory serdeFactory;
-    ResourceClaimManager claimManager;
+    private WriteAheadRepository<SerializedRepositoryRecord> wal;
+    private RepositoryRecordSerdeFactory serdeFactory;
+    private ResourceClaimManager claimManager;
 
     // WALI Provides the ability to register callbacks for when a Partition or the entire Repository is sync'ed with the underlying disk.
     // We keep track of this because we need to ensure that the ContentClaims are destroyed only after the FlowFile Repository has been
@@ -252,9 +254,9 @@ public class WriteAheadFlowFileRepository implements FlowFileRepository, SyncLis
 
         final Map<ResourceClaim, Set<ResourceClaimReference>> references = new HashMap<>();
 
-        final SnapshotCapture<RepositoryRecord> snapshot = ((SequentialAccessWriteAheadLog<RepositoryRecord>) wal).captureSnapshot();
-        for (final RepositoryRecord repositoryRecord : snapshot.getRecords().values()) {
-            final ContentClaim contentClaim = repositoryRecord.getCurrentClaim();
+        final SnapshotCapture<SerializedRepositoryRecord> snapshot = ((SequentialAccessWriteAheadLog<SerializedRepositoryRecord>) wal).captureSnapshot();
+        for (final SerializedRepositoryRecord repositoryRecord : snapshot.getRecords().values()) {
+            final ContentClaim contentClaim = repositoryRecord.getContentClaim();
             if (contentClaim == null) {
                 continue;
             }
@@ -338,14 +340,9 @@ public class WriteAheadFlowFileRepository implements FlowFileRepository, SyncLis
         };
     }
 
-    private ResourceClaimReference createResourceClaimReference(final RepositoryRecord repositoryRecord) {
-        FlowFileQueue flowFileQueue = repositoryRecord.getDestination();
-        if (flowFileQueue == null) {
-            flowFileQueue = repositoryRecord.getOriginalQueue();
-        }
-
-        final String queueIdentifier = flowFileQueue == null ? null : flowFileQueue.getIdentifier();
-        final String flowFileUuid = repositoryRecord.getCurrent().getAttribute(CoreAttributes.UUID.key());
+    private ResourceClaimReference createResourceClaimReference(final SerializedRepositoryRecord repositoryRecord) {
+        final String queueIdentifier = repositoryRecord.getQueueIdentifier();
+        final String flowFileUuid = repositoryRecord.getFlowFileRecord().getAttribute(CoreAttributes.UUID.key());
 
         return new ResourceClaimReference() {
             @Override
@@ -478,8 +475,11 @@ public class WriteAheadFlowFileRepository implements FlowFileRepository, SyncLis
             recordsForWal = Collections.emptyList();
         }
 
+        final List<SerializedRepositoryRecord> serializedRecords = new ArrayList<>(recordsForWal.size());
+        recordsForWal.forEach(record -> serializedRecords.add(new LiveSerializedRepositoryRecord(record)));
+
         // update the repository.
-        final int partitionIndex = wal.update(recordsForWal, sync);
+        final int partitionIndex = wal.update(serializedRecords, sync);
         updateContentClaims(records, partitionIndex);
     }
 
@@ -661,7 +661,10 @@ public class WriteAheadFlowFileRepository implements FlowFileRepository, SyncLis
         // We could instead have a single record with Update Type of 'SWAP OUT' and just include swap file location, Queue ID,
         // and all FlowFile ID's.
         // update WALI to indicate that the records were swapped out.
-        wal.update(repoRecords, true);
+        final List<SerializedRepositoryRecord> serializedRepositoryRecords = new ArrayList<>(repoRecords.size());
+        repoRecords.forEach(record -> serializedRepositoryRecords.add(new LiveSerializedRepositoryRecord(record)));
+
+        wal.update(serializedRepositoryRecords, true);
 
         synchronized (this.swapLocationSuffixes) {
             this.swapLocationSuffixes.add(normalizeSwapLocation(swapLocation));
@@ -708,17 +711,17 @@ public class WriteAheadFlowFileRepository implements FlowFileRepository, SyncLis
         }
     }
 
-    private Optional<Collection<RepositoryRecord>> migrateFromSequentialAccessLog(final WriteAheadRepository<RepositoryRecord> toUpdate) throws IOException {
+    private Optional<Collection<SerializedRepositoryRecord>> migrateFromSequentialAccessLog(final WriteAheadRepository<SerializedRepositoryRecord> toUpdate) throws IOException {
         final String recoveryDirName = nifiProperties.getProperty(FLOWFILE_REPOSITORY_DIRECTORY_PREFIX);
         final File recoveryDir = new File(recoveryDirName);
         if (!recoveryDir.exists()) {
             return Optional.empty();
         }
 
-        final WriteAheadRepository<RepositoryRecord> recoveryWal = new SequentialAccessWriteAheadLog<>(recoveryDir, serdeFactory, this);
+        final WriteAheadRepository<SerializedRepositoryRecord> recoveryWal = new SequentialAccessWriteAheadLog<>(recoveryDir, serdeFactory, this);
         logger.info("Encountered FlowFile Repository that was written using the Sequential Access Write Ahead Log. Will recover from this version.");
 
-        final Collection<RepositoryRecord> recordList;
+        final Collection<SerializedRepositoryRecord> recordList;
         try {
             recordList = recoveryWal.recoverRecords();
         } finally {
@@ -746,7 +749,7 @@ public class WriteAheadFlowFileRepository implements FlowFileRepository, SyncLis
     }
 
     @SuppressWarnings("deprecation")
-    private Optional<Collection<RepositoryRecord>> migrateFromMinimalLockingLog(final WriteAheadRepository<RepositoryRecord> toUpdate) throws IOException {
+    private Optional<Collection<SerializedRepositoryRecord>> migrateFromMinimalLockingLog(final WriteAheadRepository<SerializedRepositoryRecord> toUpdate) throws IOException {
         final List<File> partitionDirs = new ArrayList<>();
         for (final File recoveryFile : recoveryFiles) {
             final File[] partitions = recoveryFile.listFiles(file -> file.getName().startsWith("partition-"));
@@ -766,8 +769,8 @@ public class WriteAheadFlowFileRepository implements FlowFileRepository, SyncLis
                 .map(File::toPath)
                 .collect(Collectors.toCollection(TreeSet::new));
 
-        final Collection<RepositoryRecord> recordList;
-        final MinimalLockingWriteAheadLog<RepositoryRecord> minimalLockingWal = new MinimalLockingWriteAheadLog<>(paths, partitionDirs.size(), serdeFactory, null);
+        final Collection<SerializedRepositoryRecord> recordList;
+        final MinimalLockingWriteAheadLog<SerializedRepositoryRecord> minimalLockingWal = new MinimalLockingWriteAheadLog<>(paths, partitionDirs.size(), serdeFactory, null);
         try {
             recordList = minimalLockingWal.recoverRecords();
         } finally {
@@ -798,16 +801,45 @@ public class WriteAheadFlowFileRepository implements FlowFileRepository, SyncLis
     }
 
     @Override
-    public long loadFlowFiles(final QueueProvider queueProvider) throws IOException {
-        final Map<String, FlowFileQueue> queueMap = new HashMap<>();
-        for (final FlowFileQueue queue : queueProvider.getAllQueues()) {
-            queueMap.put(queue.getIdentifier(), queue);
+    public Set<String> findQueuesWithFlowFiles(final FlowFileSwapManager swapManager) throws IOException {
+        recoveredRecords = wal.recoverRecords();
+
+        final Set<String> queueIds = new HashSet<>();
+        for (final SerializedRepositoryRecord record : recoveredRecords) {
+            final RepositoryRecordType recordType = record.getType();
+
+            if (recordType != RepositoryRecordType.CREATE && recordType != RepositoryRecordType.UPDATE) {
+                continue;
+            }
+
+            final String queueId = record.getQueueIdentifier();
+            if (queueId != null) {
+                queueIds.add(queueId);
+            }
         }
-        serdeFactory.setQueueMap(queueMap);
 
-        // Since we used to use the MinimalLockingWriteAheadRepository, we need to ensure that if the FlowFile
-        // Repo was written using that impl, that we properly recover from the implementation.
-        Collection<RepositoryRecord> recordList = wal.recoverRecords();
+        final Set<String> recoveredSwapLocations = wal.getRecoveredSwapLocations();
+        for (final String swapLocation : recoveredSwapLocations) {
+            final String queueId = swapManager.getQueueIdentifier(swapLocation);
+            queueIds.add(queueId);
+        }
+
+        return queueIds;
+    }
+
+    @Override
+    public long loadFlowFiles(final QueueProvider queueProvider) throws IOException {
+        // If we have already loaded the records from the write-ahead logs, use them. Otherwise, recover the records now.
+        // We do this because a call to #findQueuesWithFlowFiles will recover the records, and we don't want to have to re-read
+        // the entire repository, so that method will stash the records away.
+        Collection<SerializedRepositoryRecord> recordList;
+        if (recoveredRecords == null) {
+            // Since we used to use the MinimalLockingWriteAheadRepository, we need to ensure that if the FlowFile
+            // Repo was written using that impl, that we properly recover from the implementation.
+            recordList = wal.recoverRecords();
+        } else {
+            recordList = recoveredRecords;
+        }
 
         final Set<String> recoveredSwapLocations = wal.getRecoveredSwapLocations();
         synchronized (this.swapLocationSuffixes) {
@@ -832,33 +864,44 @@ public class WriteAheadFlowFileRepository implements FlowFileRepository, SyncLis
             }
         }
 
-        serdeFactory.setQueueMap(null);
-
-        for (final RepositoryRecord record : recordList) {
-            final ContentClaim claim = record.getCurrentClaim();
-            if (claim != null) {
-                claimManager.incrementClaimantCount(claim.getResourceClaim());
-            }
+        final Map<String, FlowFileQueue> queueMap = new HashMap<>();
+        for (final FlowFileQueue queue : queueProvider.getAllQueues()) {
+            queueMap.put(queue.getIdentifier(), queue);
         }
 
-        // Determine the next sequence number for FlowFiles
         int numFlowFilesMissingQueue = 0;
         long maxId = 0;
-        for (final RepositoryRecord record : recordList) {
+        for (final SerializedRepositoryRecord record : recordList) {
             final long recordId = serdeFactory.getRecordIdentifier(record);
             if (recordId > maxId) {
                 maxId = recordId;
             }
 
-            final FlowFileRecord flowFile = record.getCurrent();
-            final FlowFileQueue queue = record.getOriginalQueue();
-            if (queue == null) {
+            final String queueId = record.getQueueIdentifier();
+            if (queueId == null) {
                 numFlowFilesMissingQueue++;
-            } else {
-                queue.put(flowFile);
+                logger.warn("Encounted Repository Record (id={}) with no Queue Identifier. Dropping this FlowFile", recordId);
+                continue;
+            }
+
+            final FlowFileQueue flowFileQueue = queueMap.get(queueId);
+            if (flowFileQueue == null) {
+                numFlowFilesMissingQueue++;
+                logger.warn("Encountered Repository Record (id={}) with Queue identifier {} but no Queue exists with that ID. Dropping this FlowFile", recordId, queueId);
+                continue;
+            }
+
+            flowFileQueue.put(record.getFlowFileRecord());
+
+            final ContentClaim claim = record.getContentClaim();
+            if (claim != null) {
+                claimManager.incrementClaimantCount(claim.getResourceClaim());
             }
         }
 
+        // If recoveredRecords has been populated it need to be nulled out now because it is no longer useful and can be garbage collected.
+        recoveredRecords = null;
+
         // Set the AtomicLong to 1 more than the max ID so that calls to #getNextFlowFileSequence() will
         // return the appropriate number.
         flowFileSequenceGenerator.set(maxId + 1);
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 bd0a7ad..26dfe39 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.StringEncryptor;
+import org.apache.nifi.services.FlowService;
 
 /**
  */
@@ -35,13 +36,14 @@ public interface FlowSynchronizer {
      * @param controller the flow controller
      * @param dataFlow the flow to load the controller with. If the flow is null or zero length, then the controller must not have a flow or else an UninheritableFlowException will be thrown.
      * @param encryptor used for the encryption/decryption of sensitive property values
+     * @param flowService the flow service
      *
      * @throws FlowSerializationException if proposed flow is not a valid flow configuration file
      * @throws UninheritableFlowException if the proposed flow cannot be loaded by the controller because in doing so would risk orphaning flow files
      * @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, StringEncryptor encryptor)
+    void sync(FlowController controller, DataFlow dataFlow, StringEncryptor encryptor, FlowService flowService)
             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/fingerprint/FingerprintFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/fingerprint/FingerprintFactory.java
index dea010a..c8f9356 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/fingerprint/FingerprintFactory.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/fingerprint/FingerprintFactory.java
@@ -16,23 +16,6 @@
  */
 package org.apache.nifi.fingerprint;
 
-import java.io.ByteArrayInputStream;
-import java.io.IOException;
-import java.io.UnsupportedEncodingException;
-import java.security.NoSuchAlgorithmException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.List;
-import java.util.Map;
-import java.util.SortedMap;
-import java.util.TreeMap;
-import java.util.stream.Stream;
-import javax.xml.XMLConstants;
-import javax.xml.parsers.DocumentBuilder;
-import javax.xml.parsers.DocumentBuilderFactory;
-import javax.xml.validation.Schema;
-import javax.xml.validation.SchemaFactory;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.bundle.BundleCoordinate;
 import org.apache.nifi.components.ConfigurableComponent;
@@ -58,6 +41,22 @@ import org.w3c.dom.Node;
 import org.w3c.dom.NodeList;
 import org.xml.sax.SAXException;
 
+import javax.xml.XMLConstants;
+import javax.xml.parsers.DocumentBuilder;
+import javax.xml.parsers.DocumentBuilderFactory;
+import javax.xml.validation.Schema;
+import javax.xml.validation.SchemaFactory;
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import java.util.stream.Stream;
+
 /**
  * <p>Creates a fingerprint of a flow.xml. The order of elements or attributes in the flow.xml does not influence the fingerprint generation.
  *
@@ -145,11 +144,7 @@ public class FingerprintFactory {
      * @throws FingerprintException if the fingerprint failed to be generated
      */
     public synchronized String createFingerprint(final byte[] flowBytes, final FlowController controller) throws FingerprintException {
-        try {
-            return createFingerprint(parseFlow(flowBytes), controller);
-        } catch (final NoSuchAlgorithmException e) {
-            throw new FingerprintException(e);
-        }
+        return createFingerprint(parseFlow(flowBytes), controller);
     }
 
     /**
@@ -158,11 +153,8 @@ public class FingerprintFactory {
      * @param flowDoc the DOM
      *
      * @return the fingerprint
-     *
-     * @throws NoSuchAlgorithmException ex
-     * @throws UnsupportedEncodingException ex
      */
-    private String createFingerprint(final Document flowDoc, final FlowController controller) throws NoSuchAlgorithmException {
+    public synchronized String createFingerprint(final Document flowDoc, final FlowController controller) {
         if (flowDoc == null) {
             return "";
         }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java
index b725d6d..e7af89c 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java
@@ -2632,7 +2632,12 @@ public final class StandardProcessGroup implements ProcessGroup {
     }
 
     @Override
-    public void verifyCanDelete(final boolean ignoreConnections) {
+    public void verifyCanDelete(final boolean ignorePortConnections) {
+        verifyCanDelete(ignorePortConnections, false);
+    }
+
+    @Override
+    public void verifyCanDelete(final boolean ignoreConnections, final boolean ignoreTemplates) {
         readLock.lock();
         try {
             for (final Port port : inputPorts.values()) {
@@ -2658,10 +2663,10 @@ public final class StandardProcessGroup implements ProcessGroup {
             for (final ProcessGroup childGroup : processGroups.values()) {
                 // For nested child groups we can ignore the input/output port
                 // connections as they will be being deleted anyway.
-                childGroup.verifyCanDelete(true);
+                childGroup.verifyCanDelete(true, ignoreTemplates);
             }
 
-            if (!templates.isEmpty()) {
+            if (!ignoreTemplates && !templates.isEmpty()) {
                 throw new IllegalStateException(String.format("Cannot delete Process Group because it contains %s Templates. The Templates must be deleted first.", templates.size()));
             }
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/logging/repository/StandardLogRepository.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/logging/repository/StandardLogRepository.java
index 3287632..dde2a8c 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/logging/repository/StandardLogRepository.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/logging/repository/StandardLogRepository.java
@@ -134,7 +134,7 @@ public class StandardLogRepository implements LogRepository {
         try {
             // ensure observer does not exists
             if (observerLookup.containsKey(observerIdentifier)) {
-                throw new IllegalStateException("The specified observer identifier already exists.");
+                throw new IllegalStateException("The specified observer identifier (" + observerIdentifier + ") already exists.");
             }
 
             final LogLevel[] allLevels = LogLevel.values();
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/persistence/FlowConfigurationDAO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/persistence/FlowConfigurationDAO.java
index 0565eb7..3a4fa21 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/persistence/FlowConfigurationDAO.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/persistence/FlowConfigurationDAO.java
@@ -16,16 +16,17 @@
  */
 package org.apache.nifi.persistence;
 
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-
 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.serialization.FlowSerializationException;
 import org.apache.nifi.controller.serialization.FlowSynchronizationException;
+import org.apache.nifi.services.FlowService;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
 
 /**
  * Interface to define service methods for FlowController configuration.
@@ -45,6 +46,7 @@ public interface FlowConfigurationDAO {
      *
      * @param controller a controller
      * @param dataFlow the flow to load
+     * @param flowService the flow service
      * @throws java.io.IOException
      *
      * @throws FlowSerializationException if proposed flow is not a valid flow configuration file
@@ -52,7 +54,7 @@ public interface FlowConfigurationDAO {
      * @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 does not exist in the controller
      */
-    void load(FlowController controller, DataFlow dataFlow)
+    void load(FlowController controller, DataFlow dataFlow, FlowService flowService)
             throws IOException, FlowSerializationException, FlowSynchronizationException, UninheritableFlowException, MissingBundleException;
 
     /**
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/persistence/StandardXMLFlowConfigurationDAO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/persistence/StandardXMLFlowConfigurationDAO.java
index e7e3ce8..074a9b7 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/persistence/StandardXMLFlowConfigurationDAO.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/persistence/StandardXMLFlowConfigurationDAO.java
@@ -16,16 +16,6 @@
  */
 package org.apache.nifi.persistence;
 
-import java.io.File;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import java.nio.file.StandardOpenOption;
-import java.util.zip.GZIPInputStream;
-import java.util.zip.GZIPOutputStream;
-
 import org.apache.nifi.cluster.protocol.DataFlow;
 import org.apache.nifi.controller.FlowController;
 import org.apache.nifi.controller.MissingBundleException;
@@ -37,11 +27,22 @@ import org.apache.nifi.controller.serialization.FlowSynchronizer;
 import org.apache.nifi.controller.serialization.StandardFlowSerializer;
 import org.apache.nifi.encrypt.StringEncryptor;
 import org.apache.nifi.nar.ExtensionManager;
+import org.apache.nifi.services.FlowService;
 import org.apache.nifi.util.NiFiProperties;
 import org.apache.nifi.util.file.FileUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.util.zip.GZIPInputStream;
+import java.util.zip.GZIPOutputStream;
+
 public final class StandardXMLFlowConfigurationDAO implements FlowConfigurationDAO {
 
     private final Path flowXmlPath;
@@ -81,11 +82,11 @@ public final class StandardXMLFlowConfigurationDAO implements FlowConfigurationD
     }
 
     @Override
-    public synchronized void load(final FlowController controller, final DataFlow dataFlow)
+    public synchronized void load(final FlowController controller, final DataFlow dataFlow, final FlowService flowService)
             throws IOException, FlowSerializationException, FlowSynchronizationException, UninheritableFlowException, MissingBundleException {
 
         final FlowSynchronizer flowSynchronizer = new StandardFlowSynchronizer(encryptor, nifiProperties, extensionManager);
-        controller.synchronize(flowSynchronizer, dataFlow);
+        controller.synchronize(flowSynchronizer, dataFlow, flowService);
 
         if (StandardFlowSynchronizer.isEmpty(dataFlow)) {
             // If the dataflow is empty, we want to save it. We do this because when we start up a brand new cluster with no
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/groovy/org/apache/nifi/controller/serialization/FlowFromDOMFactoryTest.groovy b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/groovy/org/apache/nifi/controller/serialization/FlowFromDOMFactoryTest.groovy
index 8f82d3a..a2fc4fe 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/groovy/org/apache/nifi/controller/serialization/FlowFromDOMFactoryTest.groovy
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/groovy/org/apache/nifi/controller/serialization/FlowFromDOMFactoryTest.groovy
@@ -17,14 +17,12 @@
 package org.apache.nifi.controller.serialization
 
 import org.apache.commons.codec.binary.Hex
-import org.apache.nifi.controller.StandardFlowSynchronizer
 import org.apache.nifi.encrypt.EncryptionException
 import org.apache.nifi.encrypt.StringEncryptor
 import org.apache.nifi.properties.StandardNiFiProperties
 import org.apache.nifi.security.kms.CryptoUtils
 import org.apache.nifi.security.util.EncryptionMethod
 import org.apache.nifi.util.NiFiProperties
-import org.apache.nifi.web.api.dto.ProcessGroupDTO
 import org.bouncycastle.jce.provider.BouncyCastleProvider
 import org.junit.After
 import org.junit.Before
@@ -34,15 +32,12 @@ import org.junit.runner.RunWith
 import org.junit.runners.JUnit4
 import org.slf4j.Logger
 import org.slf4j.LoggerFactory
-import org.w3c.dom.Document
-import org.w3c.dom.Element
 
 import javax.crypto.Cipher
 import javax.crypto.SecretKey
 import javax.crypto.SecretKeyFactory
 import javax.crypto.spec.PBEKeySpec
 import javax.crypto.spec.PBEParameterSpec
-import java.nio.charset.StandardCharsets
 import java.security.Security
 
 import static groovy.test.GroovyAssert.shouldFail
@@ -140,82 +135,6 @@ class FlowFromDOMFactoryTest {
         assert msg.message =~ "Check that the ${KEY} value in nifi.properties matches the value used to encrypt the flow.xml.gz file"
     }
 
-    @Test
-    void testShouldDecryptSensitiveFlowValueRegardlessOfPropertySensitiveStatus() throws Exception {
-        // Arrange
-
-        // Create a mock Element object to be parsed
-
-        // TODO: Mock call to StandardFlowSynchronizer#readFlowFromDisk()
-        final String FLOW_XML = """<?xml version="1.0" encoding="UTF-8" standalone="no"?>
-<flowController encoding-version="1.3">
-  <maxTimerDrivenThreadCount>10</maxTimerDrivenThreadCount>
-  <maxEventDrivenThreadCount>5</maxEventDrivenThreadCount>
-  <registries/>
-  <rootGroup>
-    <id>32aeba59-0167-1000-fc76-847bf5d10d73</id>
-    <name>NiFi Flow</name>
-    <position x="0.0" y="0.0"/>
-    <comment/>
-    <processor>
-      <id>32af5e4e-0167-1000-ad5f-c79ff57c851e</id>
-      <name>Example Processor</name>
-      <position x="461.0" y="80.0"/>
-      <styles/>
-      <comment/>
-      <class>org.apache.nifi.processors.test.ExampleProcessor</class>
-      <bundle>
-        <group>org.apache.nifi</group>
-        <artifact>nifi-test-nar</artifact>
-        <version>1.9.0-SNAPSHOT</version>
-      </bundle>
-      <maxConcurrentTasks>1</maxConcurrentTasks>
-      <schedulingPeriod>0 sec</schedulingPeriod>
-      <penalizationPeriod>30 sec</penalizationPeriod>
-      <yieldPeriod>1 sec</yieldPeriod>
-      <bulletinLevel>WARN</bulletinLevel>
-      <lossTolerant>false</lossTolerant>
-      <scheduledState>STOPPED</scheduledState>
-      <schedulingStrategy>TIMER_DRIVEN</schedulingStrategy>
-      <executionNode>ALL</executionNode>
-      <runDurationNanos>0</runDurationNanos>
-      <property>
-        <name>Plaintext Property</name>
-        <value>plain value</value>
-      </property>
-      <property>
-        <name>Sensitive Property</name>
-        <value>enc{29077eedc9af7515cc3e0d2d29a93a5cbb059164876948458fd0c890009c8661}</value>
-      </property>
-    </processor>
-  </rootGroup>
-  <controllerServices/>
-  <reportingTasks/>
-</flowController>
-"""
-
-        // TODO: Mock call to StandardFlowSynchronizer#parseFlowBytes()
-        Document flow = StandardFlowSynchronizer.parseFlowBytes(FLOW_XML.getBytes(StandardCharsets.UTF_8))
-
-        // Logic to extract root process group
-        final Element rootElement = flow.getDocumentElement()
-
-        final Element rootGroupElement = (Element) rootElement.getElementsByTagName("rootGroup").item(0)
-        final FlowEncodingVersion encodingVersion = FlowEncodingVersion.parse(rootGroupElement)
-
-        StringEncryptor flowEncryptor = new StringEncryptor(EncryptionMethod.MD5_128AES.algorithm, EncryptionMethod.MD5_128AES.provider, DEFAULT_PASSWORD)
-
-        // Act
-        ProcessGroupDTO decryptedProcessGroupDTO = FlowFromDOMFactory.getProcessGroup(null, rootGroupElement, flowEncryptor, encodingVersion)
-        logger.info("PG DTO: ${decryptedProcessGroupDTO}")
-
-        // Assert
-        def processorProperties = decryptedProcessGroupDTO.contents.processors.first().config.properties
-        logger.info("Parsed processor properties: ${processorProperties}")
-
-        assert processorProperties.find { it.key == "Plaintext Property" }.value == "plain value"
-        assert processorProperties.find { it.key == "Sensitive Property" }.value == "sensitive value"
-    }
 
     private
     static Cipher generateCipher(boolean encryptMode, String password = DEFAULT_PASSWORD, byte[] salt = DEFAULT_SALT, int iterationCount = DEFAULT_ITERATION_COUNT) {
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/groovy/org/apache/nifi/wali/EncryptedSequentialAccessWriteAheadLogTest.groovy b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/groovy/org/apache/nifi/wali/EncryptedSequentialAccessWriteAheadLogTest.groovy
index d653eee..07d88ba 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/groovy/org/apache/nifi/wali/EncryptedSequentialAccessWriteAheadLogTest.groovy
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/groovy/org/apache/nifi/wali/EncryptedSequentialAccessWriteAheadLogTest.groovy
@@ -21,9 +21,11 @@ import ch.qos.logback.classic.Level
 import org.apache.commons.lang3.SystemUtils
 import org.apache.nifi.controller.queue.FlowFileQueue
 import org.apache.nifi.controller.repository.EncryptedSchemaRepositoryRecordSerde
+import org.apache.nifi.controller.repository.LiveSerializedRepositoryRecord
 import org.apache.nifi.controller.repository.RepositoryRecord
 import org.apache.nifi.controller.repository.RepositoryRecordType
 import org.apache.nifi.controller.repository.SchemaRepositoryRecordSerde
+import org.apache.nifi.controller.repository.SerializedRepositoryRecord
 import org.apache.nifi.controller.repository.StandardFlowFileRecord
 import org.apache.nifi.controller.repository.StandardRepositoryRecord
 import org.apache.nifi.controller.repository.StandardRepositoryRecordSerdeFactory
@@ -63,14 +65,13 @@ class EncryptedSequentialAccessWriteAheadLogTest extends GroovyTestCase {
     public static final String TEST_QUEUE_IDENTIFIER = "testQueueIdentifier"
 
     private ResourceClaimManager claimManager
-    private Map<String, FlowFileQueue> queueMap
     private FlowFileQueue flowFileQueue
     private ByteArrayOutputStream byteArrayOutputStream
     private DataOutputStream dataOutputStream
 
     // TODO: Mock the wrapped serde
     // TODO: Make integration test with real wrapped serde
-    private SerDe<RepositoryRecord> wrappedSerDe
+    private SerDe<SerializedRepositoryRecord> wrappedSerDe
 
     private static final String KPI = STATIC_KEY_PROVIDER_CLASS_NAME
     private static final String KPL = ""
@@ -83,7 +84,7 @@ class EncryptedSequentialAccessWriteAheadLogTest extends GroovyTestCase {
 
     private EncryptedSchemaRepositoryRecordSerde esrrs
 
-    private final EncryptedSequentialAccessWriteAheadLog<RepositoryRecord> encryptedWAL
+    private final EncryptedSequentialAccessWriteAheadLog<SerializedRepositoryRecord> encryptedWAL
 
     @Rule
     public TestName testName = new TestName()
@@ -101,12 +102,10 @@ class EncryptedSequentialAccessWriteAheadLogTest extends GroovyTestCase {
     @Before
     void setUp() throws Exception {
         claimManager = new StandardResourceClaimManager()
-        queueMap = [:]
         flowFileQueue = createAndRegisterMockQueue(TEST_QUEUE_IDENTIFIER)
         byteArrayOutputStream = new ByteArrayOutputStream()
         dataOutputStream = new DataOutputStream(byteArrayOutputStream)
         wrappedSerDe = new SchemaRepositoryRecordSerde(claimManager)
-        wrappedSerDe.setQueueMap(queueMap)
 
         flowFileREC = new FlowFileRepositoryEncryptionConfiguration(KPI, KPL, KEY_ID, KEYS, REPO_IMPL)
 
@@ -116,7 +115,6 @@ class EncryptedSequentialAccessWriteAheadLogTest extends GroovyTestCase {
     @After
     void tearDown() throws Exception {
         claimManager.purge()
-        queueMap.clear()
     }
 
     private FlowFileQueue createMockQueue(String identifier = testName.methodName + new Date().toString()) {
@@ -128,16 +126,16 @@ class EncryptedSequentialAccessWriteAheadLogTest extends GroovyTestCase {
 
     private FlowFileQueue createAndRegisterMockQueue(String identifier = testName.methodName + new Date().toString()) {
         FlowFileQueue queue = createMockQueue(identifier)
-        queueMap.put(identifier, queue)
         queue
     }
 
-    private RepositoryRecord buildCreateRecord(FlowFileQueue queue, Map<String, String> attributes = [:]) {
+    private SerializedRepositoryRecord buildCreateRecord(FlowFileQueue queue, Map<String, String> attributes = [:]) {
         StandardRepositoryRecord record = new StandardRepositoryRecord(queue)
         StandardFlowFileRecord.Builder ffrb = new StandardFlowFileRecord.Builder().id(System.nanoTime())
         ffrb.addAttributes([uuid: getMockUUID()] + attributes as Map<String, String>)
         record.setWorking(ffrb.build())
-        record
+
+        return new LiveSerializedRepositoryRecord(record);
     }
 
     private String getMockUUID() {
@@ -150,12 +148,12 @@ class EncryptedSequentialAccessWriteAheadLogTest extends GroovyTestCase {
         // Arrange
         final EncryptedSchemaRepositoryRecordSerde encryptedSerde = buildEncryptedSerDe()
 
-        final SequentialAccessWriteAheadLog<RepositoryRecord> repo = createWriteRepo(encryptedSerde)
+        final SequentialAccessWriteAheadLog<SerializedRepositoryRecord> repo = createWriteRepo(encryptedSerde)
 
-        final List<RepositoryRecord> records = new ArrayList<>()
+        final List<SerializedRepositoryRecord> records = new ArrayList<>()
         10.times { int i ->
             def attributes = [name: "User ${i}" as String, age: "${i}" as String]
-            final RepositoryRecord record = buildCreateRecord(flowFileQueue, attributes)
+            final SerializedRepositoryRecord record = buildCreateRecord(flowFileQueue, attributes)
             records.add(record)
         }
 
@@ -164,15 +162,15 @@ class EncryptedSequentialAccessWriteAheadLogTest extends GroovyTestCase {
         repo.shutdown()
 
         // Assert
-        final SequentialAccessWriteAheadLog<RepositoryRecord> recoveryRepo = createRecoveryRepo()
-        final Collection<RepositoryRecord> recovered = recoveryRepo.recoverRecords()
+        final SequentialAccessWriteAheadLog<SerializedRepositoryRecord> recoveryRepo = createRecoveryRepo()
+        final Collection<SerializedRepositoryRecord> recovered = recoveryRepo.recoverRecords()
 
-        // Ensure that the same records (except now UPDATE instead of CREATE) are returned (order is not guaranteed)
+        // Ensure that the same records are returned (order is not guaranteed)
         assert recovered.size() == records.size()
-        assert recovered.every { it.type == RepositoryRecordType.UPDATE }
+        assert recovered.every { it.type == RepositoryRecordType.CREATE }
 
         // Check that all attributes (flowfile record) in the recovered records were present in the original list
-        assert recovered.every { (it as StandardRepositoryRecord).current in records*.current }
+        assert recovered.every { (it as SerializedRepositoryRecord).getFlowFileRecord() in records*.getFlowFileRecord() }
     }
 
     /** This test creates flowfile records, adds them to the repository, and then recovers them to ensure they were persisted */
@@ -181,7 +179,7 @@ class EncryptedSequentialAccessWriteAheadLogTest extends GroovyTestCase {
         // Arrange
         final EncryptedSchemaRepositoryRecordSerde encryptedSerde = buildEncryptedSerDe()
 
-        final SequentialAccessWriteAheadLog<RepositoryRecord> repo = createWriteRepo(encryptedSerde)
+        final SequentialAccessWriteAheadLog<SerializedRepositoryRecord> repo = createWriteRepo(encryptedSerde)
 
         // Turn off debugging because of the high volume
         logger.debug("Temporarily turning off DEBUG logging")
@@ -193,10 +191,10 @@ class EncryptedSequentialAccessWriteAheadLogTest extends GroovyTestCase {
         ORIGINAL_TEST_LOG_LEVEL = testLogger.getLevel()
         testLogger.setLevel(Level.INFO)
 
-        final List<RepositoryRecord> records = new ArrayList<>()
+        final List<SerializedRepositoryRecord> records = new ArrayList<>()
         100_000.times { int i ->
             def attributes = [name: "User ${i}" as String, age: "${i}" as String]
-            final RepositoryRecord record = buildCreateRecord(flowFileQueue, attributes)
+            final SerializedRepositoryRecord record = buildCreateRecord(flowFileQueue, attributes)
             records.add(record)
         }
 
@@ -205,12 +203,12 @@ class EncryptedSequentialAccessWriteAheadLogTest extends GroovyTestCase {
         repo.shutdown()
 
         // Assert
-        final SequentialAccessWriteAheadLog<RepositoryRecord> recoveryRepo = createRecoveryRepo()
-        final Collection<RepositoryRecord> recovered = recoveryRepo.recoverRecords()
+        final SequentialAccessWriteAheadLog<SerializedRepositoryRecord> recoveryRepo = createRecoveryRepo()
+        final Collection<SerializedRepositoryRecord> recovered = recoveryRepo.recoverRecords()
 
         // Ensure that the same records (except now UPDATE instead of CREATE) are returned (order is not guaranteed)
         assert recovered.size() == records.size()
-        assert recovered.every { it.type == RepositoryRecordType.UPDATE }
+        assert recovered.every { it.type == RepositoryRecordType.CREATE }
 
         // Reset log level
         encryptorLogger.setLevel(ORIGINAL_REPO_LOG_LEVEL)
@@ -221,37 +219,36 @@ class EncryptedSequentialAccessWriteAheadLogTest extends GroovyTestCase {
     private EncryptedSchemaRepositoryRecordSerde buildEncryptedSerDe(FlowFileRepositoryEncryptionConfiguration ffrec = flowFileREC) {
         final StandardRepositoryRecordSerdeFactory factory = new StandardRepositoryRecordSerdeFactory(claimManager)
         SchemaRepositoryRecordSerde wrappedSerDe = factory.createSerDe() as SchemaRepositoryRecordSerde
-        wrappedSerDe.setQueueMap(queueMap)
         return new EncryptedSchemaRepositoryRecordSerde(wrappedSerDe, ffrec)
     }
 
-    private SequentialAccessWriteAheadLog<RepositoryRecord> createWriteRepo() throws IOException {
+    private SequentialAccessWriteAheadLog<SerializedRepositoryRecord> createWriteRepo() throws IOException {
         return createWriteRepo(buildEncryptedSerDe())
     }
 
-    private SequentialAccessWriteAheadLog<RepositoryRecord> createWriteRepo(final SerDe<RepositoryRecord> serde) throws IOException {
+    private SequentialAccessWriteAheadLog<SerializedRepositoryRecord> createWriteRepo(final SerDe<SerializedRepositoryRecord> serde) throws IOException {
         final File targetDir = new File("target")
         final File storageDir = new File(targetDir, testName?.methodName ?: "unknown_test")
         deleteRecursively(storageDir)
         assertTrue(storageDir.mkdirs())
 
-        final SerDeFactory<RepositoryRecord> serdeFactory = new SingletonSerDeFactory<>(serde)
-        final SequentialAccessWriteAheadLog<RepositoryRecord> repo = new SequentialAccessWriteAheadLog<>(storageDir, serdeFactory)
+        final SerDeFactory<SerializedRepositoryRecord> serdeFactory = new SingletonSerDeFactory<>(serde)
+        final SequentialAccessWriteAheadLog<SerializedRepositoryRecord> repo = new SequentialAccessWriteAheadLog<>(storageDir, serdeFactory)
 
-        final Collection<RepositoryRecord> recovered = repo.recoverRecords()
+        final Collection<SerializedRepositoryRecord> recovered = repo.recoverRecords()
         assertNotNull(recovered)
         assertTrue(recovered.isEmpty())
 
         return repo
     }
 
-    private SequentialAccessWriteAheadLog<RepositoryRecord> createRecoveryRepo() throws IOException {
+    private SequentialAccessWriteAheadLog<SerializedRepositoryRecord> createRecoveryRepo() throws IOException {
         final File targetDir = new File("target")
         final File storageDir = new File(targetDir, testName?.methodName ?: "unknown_test")
 
-        final SerDe<RepositoryRecord> serde = buildEncryptedSerDe()
-        final SerDeFactory<RepositoryRecord> serdeFactory = new SingletonSerDeFactory<>(serde)
-        final SequentialAccessWriteAheadLog<RepositoryRecord> repo = new SequentialAccessWriteAheadLog<>(storageDir, serdeFactory)
+        final SerDe<SerializedRepositoryRecord> serde = buildEncryptedSerDe()
+        final SerDeFactory<SerializedRepositoryRecord> serdeFactory = new SingletonSerDeFactory<>(serde)
+        final SequentialAccessWriteAheadLog<SerializedRepositoryRecord> repo = new SequentialAccessWriteAheadLog<>(storageDir, serdeFactory)
 
         return repo
     }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestFlowController.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestFlowController.java
index cae87dd..2accf6a 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestFlowController.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestFlowController.java
@@ -29,6 +29,7 @@ import org.apache.nifi.authorization.User;
 import org.apache.nifi.bundle.Bundle;
 import org.apache.nifi.bundle.BundleCoordinate;
 import org.apache.nifi.cluster.protocol.DataFlow;
+import org.apache.nifi.cluster.protocol.StandardDataFlow;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.controller.exception.ProcessorInstantiationException;
 import org.apache.nifi.controller.flow.FlowManager;
@@ -59,6 +60,7 @@ import org.apache.nifi.registry.variable.FileBasedVariableRegistry;
 import org.apache.nifi.reporting.BulletinRepository;
 import org.apache.nifi.scheduling.ExecutionNode;
 import org.apache.nifi.scheduling.SchedulingStrategy;
+import org.apache.nifi.services.FlowService;
 import org.apache.nifi.util.NiFiProperties;
 import org.apache.nifi.web.api.dto.BundleDTO;
 import org.apache.nifi.web.api.dto.ControllerServiceDTO;
@@ -204,14 +206,12 @@ public class TestFlowController {
 
         // create a mock proposed data flow with the same auth fingerprint as the current authorizer
         final String authFingerprint = authorizer.getFingerprint();
-        final DataFlow proposedDataFlow = Mockito.mock(DataFlow.class);
-        when(proposedDataFlow.getAuthorizerFingerprint()).thenReturn(authFingerprint.getBytes(StandardCharsets.UTF_8));
-
         final File flowFile = new File("src/test/resources/conf/reporting-task-with-cs-flow-0.7.0.xml");
         final String flow = IOUtils.toString(new FileInputStream(flowFile), StandardCharsets.UTF_8);
-        when(proposedDataFlow.getFlow()).thenReturn(flow.getBytes(StandardCharsets.UTF_8));
 
-        controller.synchronize(standardFlowSynchronizer, proposedDataFlow);
+        final DataFlow proposedDataFlow = new StandardDataFlow(flow.getBytes(StandardCharsets.UTF_8), null, authFingerprint.getBytes(StandardCharsets.UTF_8), Collections.emptySet());
+
+        controller.synchronize(standardFlowSynchronizer, proposedDataFlow, Mockito.mock(FlowService.class));
 
         // should be two controller services
         final Set<ControllerServiceNode> controllerServiceNodes = controller.getFlowManager().getAllControllerServices();
@@ -268,37 +268,39 @@ public class TestFlowController {
 
         // create a mock proposed data flow with the same auth fingerprint as the current authorizer
         final String authFingerprint = authorizer.getFingerprint();
-        final DataFlow proposedDataFlow = Mockito.mock(DataFlow.class);
-        when(proposedDataFlow.getAuthorizerFingerprint()).thenReturn(authFingerprint.getBytes(StandardCharsets.UTF_8));
-
         final File flowFile = new File("src/test/resources/conf/processor-with-cs-flow-0.7.0.xml");
         final String flow = IOUtils.toString(new FileInputStream(flowFile), StandardCharsets.UTF_8);
-        when(proposedDataFlow.getFlow()).thenReturn(flow.getBytes(StandardCharsets.UTF_8));
-
-        controller.synchronize(standardFlowSynchronizer, proposedDataFlow);
 
-        // should be two controller services
-        final Set<ControllerServiceNode> controllerServiceNodes = controller.getFlowManager().getAllControllerServices();
-        assertNotNull(controllerServiceNodes);
-        assertEquals(1, controllerServiceNodes.size());
-
-        // find the controller service that was moved to the root group
-        final ControllerServiceNode rootGroupCs = controllerServiceNodes.stream().filter(c -> c.getProcessGroup() != null).findFirst().get();
-        assertNotNull(rootGroupCs);
+        final DataFlow proposedDataFlow = new StandardDataFlow(flow.getBytes(StandardCharsets.UTF_8), null, authFingerprint.getBytes(StandardCharsets.UTF_8), Collections.emptySet());
 
-        // should be one processor
-        final Collection<ProcessorNode> processorNodes = controller.getFlowManager().getRootGroup().getProcessors();
-        assertNotNull(processorNodes);
-        assertEquals(1, processorNodes.size());
+        controller.synchronize(standardFlowSynchronizer, proposedDataFlow, Mockito.mock(FlowService.class));
 
-        // verify the processor is still pointing at the controller service that got moved to the root group
-        final ProcessorNode processorNode = processorNodes.stream().findFirst().get();
-        final PropertyDescriptor procControllerServiceProp = processorNode.getRawPropertyValues().entrySet().stream()
+        try {
+            // should be two controller services
+            final Set<ControllerServiceNode> controllerServiceNodes = controller.getFlowManager().getAllControllerServices();
+            assertNotNull(controllerServiceNodes);
+            assertEquals(1, controllerServiceNodes.size());
+
+            // find the controller service that was moved to the root group
+            final ControllerServiceNode rootGroupCs = controllerServiceNodes.stream().filter(c -> c.getProcessGroup() != null).findFirst().get();
+            assertNotNull(rootGroupCs);
+
+            // should be one processor
+            final Collection<ProcessorNode> processorNodes = controller.getFlowManager().getRootGroup().getProcessors();
+            assertNotNull(processorNodes);
+            assertEquals(1, processorNodes.size());
+
+            // verify the processor is still pointing at the controller service that got moved to the root group
+            final ProcessorNode processorNode = processorNodes.stream().findFirst().get();
+            final PropertyDescriptor procControllerServiceProp = processorNode.getRawPropertyValues().entrySet().stream()
                 .filter(e -> e.getValue().equals(rootGroupCs.getIdentifier()))
                 .map(Map.Entry::getKey)
                 .findFirst()
                 .get();
-        assertNotNull(procControllerServiceProp);
+            assertNotNull(procControllerServiceProp);
+        } finally {
+            purgeFlow();
+        }
     }
 
     @Test
@@ -311,34 +313,100 @@ public class TestFlowController {
         final DataFlow proposedDataFlow = Mockito.mock(DataFlow.class);
         when(proposedDataFlow.getAuthorizerFingerprint()).thenReturn(authFingerprint.getBytes(StandardCharsets.UTF_8));
 
-        controller.synchronize(standardFlowSynchronizer, proposedDataFlow);
+        controller.synchronize(standardFlowSynchronizer, proposedDataFlow, Mockito.mock(FlowService.class));
 
         assertEquals(authFingerprint, authorizer.getFingerprint());
     }
 
     @Test(expected = UninheritableFlowException.class)
-    public void testSynchronizeFlowWhenAuthorizationsAreDifferent() {
+    public void testSynchronizeFlowWhenAuthorizationsAreDifferent() throws IOException {
         final FlowSynchronizer standardFlowSynchronizer = new StandardFlowSynchronizer(
                 createEncryptorFromProperties(nifiProperties), nifiProperties, extensionManager);
 
+        final File flowFile = new File("src/test/resources/conf/processor-with-cs-flow-0.7.0.xml");
+        final String flow = IOUtils.toString(new FileInputStream(flowFile), StandardCharsets.UTF_8);
+
+        final String authFingerprint = "<authorizations></authorizations>";
+        final DataFlow proposedDataFlow = new StandardDataFlow(flow.getBytes(StandardCharsets.UTF_8), null, authFingerprint.getBytes(StandardCharsets.UTF_8), Collections.emptySet());
+
+        controller.synchronize(standardFlowSynchronizer, proposedDataFlow, Mockito.mock(FlowService.class));
+        controller.initializeFlow();
+
+        try {
+            controller.synchronize(standardFlowSynchronizer, proposedDataFlow, Mockito.mock(FlowService.class));
+            assertNotEquals(authFingerprint, authorizer.getFingerprint());
+        } finally {
+            purgeFlow();
+        }
+    }
+
+    private void purgeFlow() {
+        final ProcessGroup processGroup = controller.getFlowManager().getRootGroup();
+        for (final ProcessorNode procNode : processGroup.getProcessors()) {
+            processGroup.removeProcessor(procNode);
+        }
+        for (final ControllerServiceNode serviceNode : controller.getFlowManager().getAllControllerServices()) {
+            controller.getControllerServiceProvider().removeControllerService(serviceNode);
+        }
+    }
+
+    @Test
+    public void testSynchronizeFlowWhenAuthorizationsAreDifferentAndFlowEmpty() {
+        final FlowSynchronizer standardFlowSynchronizer = new StandardFlowSynchronizer(
+            createEncryptorFromProperties(nifiProperties), nifiProperties, extensionManager);
+
         // create a mock proposed data flow with different auth fingerprint as the current authorizer
         final String authFingerprint = "<authorizations></authorizations>";
         final DataFlow proposedDataFlow = Mockito.mock(DataFlow.class);
         when(proposedDataFlow.getAuthorizerFingerprint()).thenReturn(authFingerprint.getBytes(StandardCharsets.UTF_8));
 
-        controller.synchronize(standardFlowSynchronizer, proposedDataFlow);
+        controller.synchronize(standardFlowSynchronizer, proposedDataFlow, Mockito.mock(FlowService.class));
         assertNotEquals(authFingerprint, authorizer.getFingerprint());
+
+        assertTrue(authorizer.getGroups().isEmpty());
+        assertTrue(authorizer.getUsers().isEmpty());
+        assertTrue(authorizer.getAccessPolicies().isEmpty());
     }
 
-    @Test(expected = UninheritableFlowException.class)
-    public void testSynchronizeFlowWhenProposedAuthorizationsAreNull() {
+    @Test
+    public void testSynchronizeFlowWhenProposedAuthorizationsAreNull() throws IOException {
+        final FlowSynchronizer standardFlowSynchronizer = new StandardFlowSynchronizer(
+            createEncryptorFromProperties(nifiProperties), nifiProperties, extensionManager);
+
+        final File flowFile = new File("src/test/resources/conf/processor-with-cs-flow-0.7.0.xml");
+        final String flow = IOUtils.toString(new FileInputStream(flowFile), StandardCharsets.UTF_8);
+
+        final String authFingerprint = "<authorizations></authorizations>";
+        final DataFlow proposedDataFlow = new StandardDataFlow(flow.getBytes(StandardCharsets.UTF_8), null, authFingerprint.getBytes(StandardCharsets.UTF_8), Collections.emptySet());
+        controller.synchronize(standardFlowSynchronizer, proposedDataFlow, Mockito.mock(FlowService.class));
+
+        controller.initializeFlow();
+
+        final DataFlow dataflowWithNullAuthorizations = new StandardDataFlow(flow.getBytes(StandardCharsets.UTF_8), null, null, Collections.emptySet());
+
+        try {
+            controller.synchronize(standardFlowSynchronizer, dataflowWithNullAuthorizations, Mockito.mock(FlowService.class));
+            Assert.fail("Was able to synchronize controller with null authorizations but dataflow wasn't empty");
+        } catch (final UninheritableFlowException expected) {
+            // expected
+        } finally {
+            purgeFlow();
+        }
+    }
+
+    @Test
+    public void testSynchronizeFlowWhenProposedAuthorizationsAreNullAndEmptyFlow() {
         final FlowSynchronizer standardFlowSynchronizer = new StandardFlowSynchronizer(
                 createEncryptorFromProperties(nifiProperties), nifiProperties, extensionManager);
 
         final DataFlow proposedDataFlow = Mockito.mock(DataFlow.class);
         when(proposedDataFlow.getAuthorizerFingerprint()).thenReturn(null);
 
-        controller.synchronize(standardFlowSynchronizer, proposedDataFlow);
+        controller.synchronize(standardFlowSynchronizer, proposedDataFlow, Mockito.mock(FlowService.class));
+
+        assertTrue(authorizer.getGroups().isEmpty());
+        assertTrue(authorizer.getUsers().isEmpty());
+        assertTrue(authorizer.getAccessPolicies().isEmpty());
     }
 
     /**
@@ -374,7 +442,7 @@ public class TestFlowController {
         controller.shutdown(true);
         controller = FlowController.createStandaloneInstance(flowFileEventRepo, nifiProperties, authorizer,
             auditService, encryptor, bulletinRepo, variableRegistry, Mockito.mock(FlowRegistryClient.class), extensionManager);
-        controller.synchronize(standardFlowSynchronizer, proposedDataFlow);
+        controller.synchronize(standardFlowSynchronizer, proposedDataFlow, Mockito.mock(FlowService.class));
         assertEquals(authFingerprint, authorizer.getFingerprint());
     }
 
@@ -391,7 +459,7 @@ public class TestFlowController {
         when(proposedDataFlow.getMissingComponents()).thenReturn(missingComponents);
 
         try {
-            controller.synchronize(standardFlowSynchronizer, proposedDataFlow);
+            controller.synchronize(standardFlowSynchronizer, proposedDataFlow, Mockito.mock(FlowService.class));
             Assert.fail("Should have thrown exception");
         } catch (UninheritableFlowException e) {
             assertTrue(e.getMessage().contains("Proposed flow has missing components that are not considered missing in the current flow (1,2)"));
@@ -436,7 +504,7 @@ public class TestFlowController {
         when(proposedDataFlow.getMissingComponents()).thenReturn(new HashSet<>());
 
         try {
-            standardFlowSynchronizer.sync(mockFlowController, proposedDataFlow, stringEncryptor);
+            standardFlowSynchronizer.sync(mockFlowController, proposedDataFlow, stringEncryptor, Mockito.mock(FlowService.class));
             Assert.fail("Should have thrown exception");
         } catch (UninheritableFlowException e) {
             assertTrue(e.getMessage(), e.getMessage().contains("Current flow has missing components that are not considered missing in the proposed flow (1,2,3)"));
@@ -466,6 +534,8 @@ public class TestFlowController {
         // first sync should work because we are syncing to an empty flow controller
         syncFlow("src/test/resources/nifi/fingerprint/flow4.xml", standardFlowSynchronizer);
 
+        controller.initializeFlow();
+
         // second sync should fail because the bundle of the processor is different
         try {
             syncFlow("src/test/resources/nifi/fingerprint/flow4-with-different-bundle.xml", standardFlowSynchronizer);
@@ -482,17 +552,16 @@ public class TestFlowController {
         }
         assertNotNull(flowString);
 
-        final DataFlow proposedDataFlow1 = Mockito.mock(DataFlow.class);
-        when(proposedDataFlow1.getFlow()).thenReturn(flowString.getBytes(StandardCharsets.UTF_8));
-
+        final byte[] flowBytes = flowString.getBytes(StandardCharsets.UTF_8);
         final String authFingerprint = authorizer.getFingerprint();
-        when(proposedDataFlow1.getAuthorizerFingerprint()).thenReturn(authFingerprint.getBytes(StandardCharsets.UTF_8));
+        final byte[] authFingerprintBytes = authFingerprint.getBytes(StandardCharsets.UTF_8);
+        final DataFlow proposedDataFlow1 = new StandardDataFlow(flowBytes, null, authFingerprintBytes, Collections.emptySet());
 
-        controller.synchronize(standardFlowSynchronizer, proposedDataFlow1);
+        controller.synchronize(standardFlowSynchronizer, proposedDataFlow1, Mockito.mock(FlowService.class));
     }
 
     @Test
-    public void testCreateMissingProcessor() throws ProcessorInstantiationException {
+    public void testCreateMissingProcessor() {
         final ProcessorNode procNode = controller.getFlowManager().createProcessor("org.apache.nifi.NonExistingProcessor", "1234-Processor",
                 systemBundle.getBundleDetails().getCoordinate());
         assertNotNull(procNode);
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/SchemaRepositoryRecordSerdeTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/SchemaRepositoryRecordSerdeTest.java
index 59b0e7b..96d595a 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/SchemaRepositoryRecordSerdeTest.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/SchemaRepositoryRecordSerdeTest.java
@@ -42,7 +42,6 @@ public class SchemaRepositoryRecordSerdeTest {
     public static final String TEST_QUEUE_IDENTIFIER = "testQueueIdentifier";
     private StandardResourceClaimManager resourceClaimManager;
     private SchemaRepositoryRecordSerde schemaRepositoryRecordSerde;
-    private Map<String, FlowFileQueue> queueMap;
     private FlowFileQueue flowFileQueue;
     private ByteArrayOutputStream byteArrayOutputStream;
     private DataOutputStream dataOutputStream;
@@ -51,8 +50,6 @@ public class SchemaRepositoryRecordSerdeTest {
     public void setup() {
         resourceClaimManager = new StandardResourceClaimManager();
         schemaRepositoryRecordSerde = new SchemaRepositoryRecordSerde(resourceClaimManager);
-        queueMap = new HashMap<>();
-        schemaRepositoryRecordSerde.setQueueMap(queueMap);
         flowFileQueue = createMockQueue(TEST_QUEUE_IDENTIFIER);
         byteArrayOutputStream = new ByteArrayOutputStream();
         dataOutputStream = new DataOutputStream(byteArrayOutputStream);
@@ -72,13 +69,13 @@ public class SchemaRepositoryRecordSerdeTest {
             stringBuilder.append('a');
         }
         attributes.put(stringBuilder.toString(), "testValue");
-        schemaRepositoryRecordSerde.serializeRecord(createCreateFlowFileRecord(attributes), dataOutputStream,
+        schemaRepositoryRecordSerde.serializeRecord(new LiveSerializedRepositoryRecord(createCreateFlowFileRecord(attributes)), dataOutputStream,
                 RepositoryRecordSchema.CREATE_OR_UPDATE_SCHEMA_V1, RepositoryRecordSchema.REPOSITORY_RECORD_SCHEMA_V1);
 
         DataInputStream dataInputStream = createDataInputStream();
         schemaRepositoryRecordSerde.readHeader(dataInputStream);
-        RepositoryRecord repositoryRecord = schemaRepositoryRecordSerde.deserializeRecord(dataInputStream, 2);
-        assertNotEquals(attributes, repositoryRecord.getCurrent().getAttributes());
+        SerializedRepositoryRecord repositoryRecord = schemaRepositoryRecordSerde.deserializeRecord(dataInputStream, 2);
+        assertNotEquals(attributes, repositoryRecord.getFlowFileRecord().getAttributes());
     }
 
     @Test
@@ -90,13 +87,13 @@ public class SchemaRepositoryRecordSerdeTest {
             stringBuilder.append('a');
         }
         attributes.put("testName", stringBuilder.toString());
-        schemaRepositoryRecordSerde.serializeRecord(createCreateFlowFileRecord(attributes), dataOutputStream,
+        schemaRepositoryRecordSerde.serializeRecord(new LiveSerializedRepositoryRecord(createCreateFlowFileRecord(attributes)), dataOutputStream,
                 RepositoryRecordSchema.CREATE_OR_UPDATE_SCHEMA_V1, RepositoryRecordSchema.REPOSITORY_RECORD_SCHEMA_V1);
 
         DataInputStream dataInputStream = createDataInputStream();
         schemaRepositoryRecordSerde.readHeader(dataInputStream);
-        RepositoryRecord repositoryRecord = schemaRepositoryRecordSerde.deserializeRecord(dataInputStream, 2);
-        assertNotEquals(attributes, repositoryRecord.getCurrent().getAttributes());
+        SerializedRepositoryRecord repositoryRecord = schemaRepositoryRecordSerde.deserializeRecord(dataInputStream, 2);
+        assertNotEquals(attributes, repositoryRecord.getFlowFileRecord().getAttributes());
     }
 
     @Test
@@ -108,12 +105,12 @@ public class SchemaRepositoryRecordSerdeTest {
             stringBuilder.append('a');
         }
         attributes.put(stringBuilder.toString(), "testValue");
-        schemaRepositoryRecordSerde.serializeRecord(createCreateFlowFileRecord(attributes), dataOutputStream);
+        schemaRepositoryRecordSerde.serializeRecord(new LiveSerializedRepositoryRecord(createCreateFlowFileRecord(attributes)), dataOutputStream);
 
         DataInputStream dataInputStream = createDataInputStream();
         schemaRepositoryRecordSerde.readHeader(dataInputStream);
-        RepositoryRecord repositoryRecord = schemaRepositoryRecordSerde.deserializeRecord(dataInputStream, 2);
-        assertEquals(attributes, repositoryRecord.getCurrent().getAttributes());
+        SerializedRepositoryRecord repositoryRecord = schemaRepositoryRecordSerde.deserializeRecord(dataInputStream, 2);
+        assertEquals(attributes, repositoryRecord.getFlowFileRecord().getAttributes());
     }
 
     @Test
@@ -125,12 +122,12 @@ public class SchemaRepositoryRecordSerdeTest {
             stringBuilder.append('a');
         }
         attributes.put("testName", stringBuilder.toString());
-        schemaRepositoryRecordSerde.serializeRecord(createCreateFlowFileRecord(attributes), dataOutputStream);
+        schemaRepositoryRecordSerde.serializeRecord(new LiveSerializedRepositoryRecord(createCreateFlowFileRecord(attributes)), dataOutputStream);
 
         DataInputStream dataInputStream = createDataInputStream();
         schemaRepositoryRecordSerde.readHeader(dataInputStream);
-        RepositoryRecord repositoryRecord = schemaRepositoryRecordSerde.deserializeRecord(dataInputStream, 2);
-        assertEquals(attributes, repositoryRecord.getCurrent().getAttributes());
+        SerializedRepositoryRecord repositoryRecord = schemaRepositoryRecordSerde.deserializeRecord(dataInputStream, 2);
+        assertEquals(attributes, repositoryRecord.getFlowFileRecord().getAttributes());
     }
 
     @Test
@@ -138,13 +135,13 @@ public class SchemaRepositoryRecordSerdeTest {
         RepositoryRecordSchema.REPOSITORY_RECORD_SCHEMA_V1.writeTo(dataOutputStream);
         Map<String, String> attributes = new HashMap<>();
         attributes.put("testName", "testValue");
-        schemaRepositoryRecordSerde.serializeRecord(createCreateFlowFileRecord(attributes), dataOutputStream,
+        schemaRepositoryRecordSerde.serializeRecord(new LiveSerializedRepositoryRecord(createCreateFlowFileRecord(attributes)), dataOutputStream,
                 RepositoryRecordSchema.CREATE_OR_UPDATE_SCHEMA_V1, RepositoryRecordSchema.REPOSITORY_RECORD_SCHEMA_V1);
 
         DataInputStream dataInputStream = createDataInputStream();
         schemaRepositoryRecordSerde.readHeader(dataInputStream);
-        RepositoryRecord repositoryRecord = schemaRepositoryRecordSerde.deserializeRecord(dataInputStream, 2);
-        assertEquals(attributes, repositoryRecord.getCurrent().getAttributes());
+        SerializedRepositoryRecord repositoryRecord = schemaRepositoryRecordSerde.deserializeRecord(dataInputStream, 2);
+        assertEquals(attributes, repositoryRecord.getFlowFileRecord().getAttributes());
     }
 
     @Test
@@ -159,12 +156,13 @@ public class SchemaRepositoryRecordSerdeTest {
         StandardRepositoryRecord record = createCreateFlowFileRecord(attributes);
         record.setSwapLocation("fake");
         assertEquals(SWAP_IN, record.getType());
-        schemaRepositoryRecordSerde.serializeRecord(record, dataOutputStream, RepositoryRecordSchema.SWAP_IN_SCHEMA_V1, RepositoryRecordSchema.REPOSITORY_RECORD_SCHEMA_V1);
+        schemaRepositoryRecordSerde.serializeRecord(new LiveSerializedRepositoryRecord(record), dataOutputStream, RepositoryRecordSchema.SWAP_IN_SCHEMA_V1,
+            RepositoryRecordSchema.REPOSITORY_RECORD_SCHEMA_V1);
 
         DataInputStream dataInputStream = createDataInputStream();
         schemaRepositoryRecordSerde.readHeader(dataInputStream);
-        RepositoryRecord repositoryRecord = schemaRepositoryRecordSerde.deserializeRecord(dataInputStream, 2);
-        assertNotEquals(attributes, repositoryRecord.getCurrent().getAttributes());
+        SerializedRepositoryRecord repositoryRecord = schemaRepositoryRecordSerde.deserializeRecord(dataInputStream, 2);
+        assertNotEquals(attributes, repositoryRecord.getFlowFileRecord().getAttributes());
     }
 
     @Test
@@ -179,12 +177,13 @@ public class SchemaRepositoryRecordSerdeTest {
         StandardRepositoryRecord record = createCreateFlowFileRecord(attributes);
         record.setSwapLocation("fake");
         assertEquals(SWAP_IN, record.getType());
-        schemaRepositoryRecordSerde.serializeRecord(record, dataOutputStream, RepositoryRecordSchema.SWAP_IN_SCHEMA_V1, RepositoryRecordSchema.REPOSITORY_RECORD_SCHEMA_V1);
+        schemaRepositoryRecordSerde.serializeRecord(new LiveSerializedRepositoryRecord(record), dataOutputStream, RepositoryRecordSchema.SWAP_IN_SCHEMA_V1,
+            RepositoryRecordSchema.REPOSITORY_RECORD_SCHEMA_V1);
 
         DataInputStream dataInputStream = createDataInputStream();
         schemaRepositoryRecordSerde.readHeader(dataInputStream);
-        RepositoryRecord repositoryRecord = schemaRepositoryRecordSerde.deserializeRecord(dataInputStream, 2);
-        assertNotEquals(attributes, repositoryRecord.getCurrent().getAttributes());
+        SerializedRepositoryRecord repositoryRecord = schemaRepositoryRecordSerde.deserializeRecord(dataInputStream, 2);
+        assertNotEquals(attributes, repositoryRecord.getFlowFileRecord().getAttributes());
     }
 
     @Test
@@ -199,12 +198,12 @@ public class SchemaRepositoryRecordSerdeTest {
         StandardRepositoryRecord record = createCreateFlowFileRecord(attributes);
         record.setSwapLocation("fake");
         assertEquals(SWAP_IN, record.getType());
-        schemaRepositoryRecordSerde.serializeRecord(record, dataOutputStream);
+        schemaRepositoryRecordSerde.serializeRecord(new LiveSerializedRepositoryRecord(record), dataOutputStream);
 
         DataInputStream dataInputStream = createDataInputStream();
         schemaRepositoryRecordSerde.readHeader(dataInputStream);
-        RepositoryRecord repositoryRecord = schemaRepositoryRecordSerde.deserializeRecord(dataInputStream, 2);
-        assertEquals(attributes, repositoryRecord.getCurrent().getAttributes());
+        SerializedRepositoryRecord repositoryRecord = schemaRepositoryRecordSerde.deserializeRecord(dataInputStream, 2);
+        assertEquals(attributes, repositoryRecord.getFlowFileRecord().getAttributes());
     }
 
     @Test
@@ -219,12 +218,12 @@ public class SchemaRepositoryRecordSerdeTest {
         StandardRepositoryRecord record = createCreateFlowFileRecord(attributes);
         record.setSwapLocation("fake");
         assertEquals(SWAP_IN, record.getType());
-        schemaRepositoryRecordSerde.serializeRecord(record, dataOutputStream);
+        schemaRepositoryRecordSerde.serializeRecord(new LiveSerializedRepositoryRecord(record), dataOutputStream);
 
         DataInputStream dataInputStream = createDataInputStream();
         schemaRepositoryRecordSerde.readHeader(dataInputStream);
-        RepositoryRecord repositoryRecord = schemaRepositoryRecordSerde.deserializeRecord(dataInputStream, 2);
-        assertEquals(attributes, repositoryRecord.getCurrent().getAttributes());
+        SerializedRepositoryRecord repositoryRecord = schemaRepositoryRecordSerde.deserializeRecord(dataInputStream, 2);
+        assertEquals(attributes, repositoryRecord.getFlowFileRecord().getAttributes());
     }
 
     @Test
@@ -235,12 +234,13 @@ public class SchemaRepositoryRecordSerdeTest {
         StandardRepositoryRecord record = createCreateFlowFileRecord(attributes);
         record.setSwapLocation("fake");
         assertEquals(SWAP_IN, record.getType());
-        schemaRepositoryRecordSerde.serializeRecord(record, dataOutputStream, RepositoryRecordSchema.SWAP_IN_SCHEMA_V1, RepositoryRecordSchema.REPOSITORY_RECORD_SCHEMA_V1);
+        schemaRepositoryRecordSerde.serializeRecord(new LiveSerializedRepositoryRecord(record), dataOutputStream, RepositoryRecordSchema.SWAP_IN_SCHEMA_V1,
+            RepositoryRecordSchema.REPOSITORY_RECORD_SCHEMA_V1);
 
         DataInputStream dataInputStream = createDataInputStream();
         schemaRepositoryRecordSerde.readHeader(dataInputStream);
-        RepositoryRecord repositoryRecord = schemaRepositoryRecordSerde.deserializeRecord(dataInputStream, 2);
-        assertEquals(attributes, repositoryRecord.getCurrent().getAttributes());
+        SerializedRepositoryRecord repositoryRecord = schemaRepositoryRecordSerde.deserializeRecord(dataInputStream, 2);
+        assertEquals(attributes, repositoryRecord.getFlowFileRecord().getAttributes());
         assertEquals(SWAP_IN, repositoryRecord.getType());
     }
 
@@ -260,7 +260,6 @@ public class SchemaRepositoryRecordSerdeTest {
     private FlowFileQueue createMockQueue(String identifier) {
         FlowFileQueue flowFileQueue = mock(FlowFileQueue.class);
         when(flowFileQueue.getIdentifier()).thenReturn(identifier);
-        queueMap.put(identifier, flowFileQueue);
         return flowFileQueue;
     }
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestStandardProcessSession.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestStandardProcessSession.java
index 5e3b1b6..21be152 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestStandardProcessSession.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestStandardProcessSession.java
@@ -2403,6 +2403,11 @@ public class TestStandardProcessSession {
         }
 
         @Override
+        public Set<String> findQueuesWithFlowFiles(final FlowFileSwapManager flowFileSwapManager) throws IOException {
+            return Collections.emptySet();
+        }
+
+        @Override
         public void swapFlowFilesIn(String swapLocation, List<FlowFileRecord> flowFileRecords, FlowFileQueue queue) throws IOException {
         }
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestWriteAheadFlowFileRepository.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestWriteAheadFlowFileRepository.java
index ca656d0..c4acf9a 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestWriteAheadFlowFileRepository.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestWriteAheadFlowFileRepository.java
@@ -324,8 +324,8 @@ public class TestWriteAheadFlowFileRepository {
 
         final ResourceClaimManager claimManager = new StandardResourceClaimManager();
         final StandardRepositoryRecordSerdeFactory serdeFactory = new StandardRepositoryRecordSerdeFactory(claimManager);
-        final WriteAheadRepository<RepositoryRecord> repo = new MinimalLockingWriteAheadLog<>(path, numPartitions, serdeFactory, null);
-        final Collection<RepositoryRecord> initialRecs = repo.recoverRecords();
+        final WriteAheadRepository<SerializedRepositoryRecord> repo = new MinimalLockingWriteAheadLog<>(path, numPartitions, serdeFactory, null);
+        final Collection<SerializedRepositoryRecord> initialRecs = repo.recoverRecords();
         assertTrue(initialRecs.isEmpty());
 
         final int updateCountPerThread = totalUpdates / numThreads;
@@ -336,7 +336,7 @@ public class TestWriteAheadFlowFileRepository {
                 final Thread t = new Thread(new Runnable() {
                     @Override
                     public void run() {
-                        final List<RepositoryRecord> records = new ArrayList<>();
+                        final List<SerializedRepositoryRecord> records = new ArrayList<>();
                         final int numBatches = updateCountPerThread / batchSize;
                         final MockFlowFile baseFlowFile = new MockFlowFile(0L);
 
@@ -351,7 +351,7 @@ public class TestWriteAheadFlowFileRepository {
                                 final Map<String, String> updatedAttrs = Collections.singletonMap("uuid", uuid);
                                 record.setWorking(flowFile, updatedAttrs);
 
-                                records.add(record);
+                                records.add(new LiveSerializedRepositoryRecord(record));
                             }
 
                             try {
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/mock/MockProcessGroup.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/mock/MockProcessGroup.java
index d1dc32c..2955838 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/mock/MockProcessGroup.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/mock/MockProcessGroup.java
@@ -566,6 +566,11 @@ public class MockProcessGroup implements ProcessGroup {
     }
 
     @Override
+    public void verifyCanDelete(final boolean ignorePortConnections, final boolean ignoreTemplates) {
+
+    }
+
+    @Override
     public void verifyCanStart() {
 
     }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/FrameworkIntegrationTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/FrameworkIntegrationTest.java
index 8a01779..ca053a0 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/FrameworkIntegrationTest.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/FrameworkIntegrationTest.java
@@ -96,6 +96,7 @@ import org.apache.nifi.registry.flow.FlowRegistryClient;
 import org.apache.nifi.registry.flow.StandardFlowRegistryClient;
 import org.apache.nifi.reporting.BulletinRepository;
 import org.apache.nifi.scheduling.SchedulingStrategy;
+import org.apache.nifi.services.FlowService;
 import org.apache.nifi.util.FileUtils;
 import org.apache.nifi.util.NiFiProperties;
 import org.junit.After;
@@ -345,7 +346,7 @@ public class FrameworkIntegrationTest {
 
         // Reload the flow
         final FlowSynchronizer flowSynchronizer = new StandardFlowSynchronizer(flowController.getEncryptor(), nifiProperties, extensionManager);
-        flowController.synchronize(flowSynchronizer, new StandardDataFlow(flowBytes, null, null, Collections.emptySet()));
+        flowController.synchronize(flowSynchronizer, new StandardDataFlow(flowBytes, null, null, Collections.emptySet()), Mockito.mock(FlowService.class));
 
         // Reload FlowFiles / initialize flow
         final ProcessGroup newRootGroup = flowController.getFlowManager().getRootGroup();
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/flowfilerepo/OOMEWriteAheadFlowFileRepository.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/flowfilerepo/OOMEWriteAheadFlowFileRepository.java
index 1509e41..320db25 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/flowfilerepo/OOMEWriteAheadFlowFileRepository.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/flowfilerepo/OOMEWriteAheadFlowFileRepository.java
@@ -16,9 +16,8 @@
  */
 package org.apache.nifi.integration.flowfilerepo;
 
-import org.apache.nifi.controller.queue.FlowFileQueue;
-import org.apache.nifi.controller.repository.RepositoryRecord;
 import org.apache.nifi.controller.repository.RepositoryRecordSerdeFactory;
+import org.apache.nifi.controller.repository.SerializedRepositoryRecord;
 import org.apache.nifi.controller.repository.StandardRepositoryRecordSerdeFactory;
 import org.apache.nifi.controller.repository.WriteAheadFlowFileRepository;
 import org.apache.nifi.controller.repository.claim.ResourceClaimManager;
@@ -54,45 +53,40 @@ public class OOMEWriteAheadFlowFileRepository extends WriteAheadFlowFileReposito
         }
 
         @Override
-        public void setQueueMap(final Map<String, FlowFileQueue> queueMap) {
-            factory.setQueueMap(queueMap);
-        }
-
-        @Override
-        public SerDe<RepositoryRecord> createSerDe(final String encodingName) {
-            final SerDe<RepositoryRecord> serde = factory.createSerDe(encodingName);
+        public SerDe<SerializedRepositoryRecord> createSerDe(final String encodingName) {
+            final SerDe<SerializedRepositoryRecord> serde = factory.createSerDe(encodingName);
             return new ThrowOOMESerde(serde, 3);
         }
 
         @Override
-        public Long getRecordIdentifier(final RepositoryRecord record) {
+        public Long getRecordIdentifier(final SerializedRepositoryRecord record) {
             return factory.getRecordIdentifier(record);
         }
 
         @Override
-        public UpdateType getUpdateType(final RepositoryRecord record) {
+        public UpdateType getUpdateType(final SerializedRepositoryRecord record) {
             return factory.getUpdateType(record);
         }
 
         @Override
-        public String getLocation(final RepositoryRecord record) {
+        public String getLocation(final SerializedRepositoryRecord record) {
             return factory.getLocation(record);
         }
     }
 
 
-    private static class ThrowOOMESerde implements SerDe<RepositoryRecord> {
-        private final SerDe<RepositoryRecord> serde;
+    private static class ThrowOOMESerde implements SerDe<SerializedRepositoryRecord> {
+        private final SerDe<SerializedRepositoryRecord> serde;
         private final int afterSuccessfulAttempts;
         private int successfulUpdates = 0;
 
-        public ThrowOOMESerde(final SerDe<RepositoryRecord> serde, final int afterSuccessfulAttempts) {
+        public ThrowOOMESerde(final SerDe<SerializedRepositoryRecord> serde, final int afterSuccessfulAttempts) {
             this.serde = serde;
             this.afterSuccessfulAttempts = afterSuccessfulAttempts;
         }
 
         @Override
-        public void serializeEdit(final RepositoryRecord previousRecordState, final RepositoryRecord newRecordState, final DataOutputStream out) throws IOException {
+        public void serializeEdit(final SerializedRepositoryRecord previousRecordState, final SerializedRepositoryRecord newRecordState, final DataOutputStream out) throws IOException {
             if (successfulUpdates++ == afterSuccessfulAttempts) {
                 throw new OutOfMemoryError("Intentional OOME for unit test");
             }
@@ -101,7 +95,7 @@ public class OOMEWriteAheadFlowFileRepository extends WriteAheadFlowFileReposito
         }
 
         @Override
-        public void serializeRecord(final RepositoryRecord record, final DataOutputStream out) throws IOException {
+        public void serializeRecord(final SerializedRepositoryRecord record, final DataOutputStream out) throws IOException {
             if (successfulUpdates++ == afterSuccessfulAttempts) {
                 throw new OutOfMemoryError("Intentional OOME for unit test");
             }
@@ -110,27 +104,27 @@ public class OOMEWriteAheadFlowFileRepository extends WriteAheadFlowFileReposito
         }
 
         @Override
-        public RepositoryRecord deserializeEdit(final DataInputStream in, final Map<Object, RepositoryRecord> currentRecordStates, final int version) throws IOException {
+        public SerializedRepositoryRecord deserializeEdit(final DataInputStream in, final Map<Object, SerializedRepositoryRecord> currentRecordStates, final int version) throws IOException {
             return serde.deserializeEdit(in, currentRecordStates, version);
         }
 
         @Override
-        public RepositoryRecord deserializeRecord(final DataInputStream in, final int version) throws IOException {
+        public SerializedRepositoryRecord deserializeRecord(final DataInputStream in, final int version) throws IOException {
             return serde.deserializeRecord(in, version);
         }
 
         @Override
-        public Object getRecordIdentifier(final RepositoryRecord record) {
+        public Object getRecordIdentifier(final SerializedRepositoryRecord record) {
             return serde.getRecordIdentifier(record);
         }
 
         @Override
-        public UpdateType getUpdateType(final RepositoryRecord record) {
+        public UpdateType getUpdateType(final SerializedRepositoryRecord record) {
             return serde.getUpdateType(record);
         }
 
         @Override
-        public String getLocation(final RepositoryRecord record) {
+        public String getLocation(final SerializedRepositoryRecord record) {
             return serde.getLocation(record);
         }
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-mock-authorizer/src/main/java/org/apache/nifi/authorization/MockPolicyBasedAuthorizer.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-mock-authorizer/src/main/java/org/apache/nifi/authorization/MockPolicyBasedAuthorizer.java
index b2a9662..ccc068d 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-mock-authorizer/src/main/java/org/apache/nifi/authorization/MockPolicyBasedAuthorizer.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-mock-authorizer/src/main/java/org/apache/nifi/authorization/MockPolicyBasedAuthorizer.java
@@ -33,7 +33,7 @@ public class MockPolicyBasedAuthorizer extends AbstractPolicyBasedAuthorizer imp
     private Set<User> users = new HashSet<>();
     private Set<AccessPolicy> policies = new HashSet<>();
 
-    private Set<AuthorizationRequest> audited = new HashSet();
+    private Set<AuthorizationRequest> audited = new HashSet<>();
 
     public MockPolicyBasedAuthorizer() {
 
@@ -193,4 +193,14 @@ public class MockPolicyBasedAuthorizer extends AbstractPolicyBasedAuthorizer imp
 
     }
 
+    @Override
+    public void backupPoliciesUsersAndGroups() {
+    }
+
+    @Override
+    public void purgePoliciesUsersAndGroups() {
+        groups.clear();
+        users.clear();
+        policies.clear();
+    }
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-optimistic-locking/src/main/java/org/apache/nifi/web/revision/NaiveRevisionManager.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-optimistic-locking/src/main/java/org/apache/nifi/web/revision/NaiveRevisionManager.java
index 6f07bdb..df6c1ae 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-optimistic-locking/src/main/java/org/apache/nifi/web/revision/NaiveRevisionManager.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-optimistic-locking/src/main/java/org/apache/nifi/web/revision/NaiveRevisionManager.java
@@ -17,6 +17,12 @@
 
 package org.apache.nifi.web.revision;
 
+import org.apache.nifi.authorization.user.NiFiUser;
+import org.apache.nifi.web.InvalidRevisionException;
+import org.apache.nifi.web.Revision;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
@@ -24,12 +30,6 @@ import java.util.Objects;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 
-import org.apache.nifi.authorization.user.NiFiUser;
-import org.apache.nifi.web.InvalidRevisionException;
-import org.apache.nifi.web.Revision;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 /**
  * <p>
  * This class implements a naive approach for Revision Management.
@@ -48,6 +48,9 @@ public class NaiveRevisionManager implements RevisionManager {
     @Override
     public void reset(final Collection<Revision> revisions) {
         synchronized (this) { // avoid allowing two threads to reset versions concurrently
+            logger.info("Resetting Revisions for all components. {} revisions will be removed, {} will be added", revisionMap.size(), revisions.size());
+            logger.debug("New Revisions: {}", revisions);
+
             revisionMap.clear();
 
             for (final Revision revision : revisions) {
diff --git a/nifi-nar-bundles/nifi-ranger-bundle/nifi-ranger-plugin/src/main/java/org/apache/nifi/ranger/authorization/ManagedRangerAuthorizer.java b/nifi-nar-bundles/nifi-ranger-bundle/nifi-ranger-plugin/src/main/java/org/apache/nifi/ranger/authorization/ManagedRangerAuthorizer.java
index 7c16a80..45bdb1f 100644
--- a/nifi-nar-bundles/nifi-ranger-bundle/nifi-ranger-plugin/src/main/java/org/apache/nifi/ranger/authorization/ManagedRangerAuthorizer.java
+++ b/nifi-nar-bundles/nifi-ranger-bundle/nifi-ranger-plugin/src/main/java/org/apache/nifi/ranger/authorization/ManagedRangerAuthorizer.java
@@ -171,6 +171,15 @@ public class ManagedRangerAuthorizer extends RangerNiFiAuthorizer implements Man
     }
 
     @Override
+    public void forciblyInheritFingerprint(final String fingerprint) throws AuthorizationAccessException {
+        final String userGroupFingerprint = parseFingerprint(fingerprint);
+
+        if (userGroupProvider instanceof ConfigurableUserGroupProvider) {
+            ((ConfigurableUserGroupProvider) userGroupProvider).forciblyInheritFingerprint(userGroupFingerprint);
+        }
+    }
+
+    @Override
     public void checkInheritability(String proposedFingerprint) throws AuthorizationAccessException, UninheritableAuthorizationsException {
         final String userGroupFingerprint = parseFingerprint(proposedFingerprint);
 
@@ -183,7 +192,7 @@ public class ManagedRangerAuthorizer extends RangerNiFiAuthorizer implements Man
         }
     }
 
-    private final String parseFingerprint(final String fingerprint) throws AuthorizationAccessException {
+    private String parseFingerprint(final String fingerprint) throws AuthorizationAccessException {
         final byte[] fingerprintBytes = fingerprint.getBytes(StandardCharsets.UTF_8);
 
         try (final ByteArrayInputStream in = new ByteArrayInputStream(fingerprintBytes)) {
diff --git a/nifi-system-tests/nifi-system-test-suite/pom.xml b/nifi-system-tests/nifi-system-test-suite/pom.xml
index 9ee7780..426f506 100644
--- a/nifi-system-tests/nifi-system-test-suite/pom.xml
+++ b/nifi-system-tests/nifi-system-test-suite/pom.xml
@@ -56,6 +56,20 @@
                     </execution>
                 </executions>
             </plugin>
+            <plugin>
+            <groupId>org.apache.rat</groupId>
+                <artifactId>apache-rat-plugin</artifactId>
+                <configuration>
+                    <excludes combine.children="append">
+                        <exclude>src/test/resources/flows/mismatched-flows/flow1.xml.gz</exclude>
+                        <exclude>src/test/resources/flows/mismatched-flows/flow2.xml.gz</exclude>
+                        <exclude>src/test/resources/flows/missing-connection/with-connection.xml.gz</exclude>
+                        <exclude>src/test/resources/flows/missing-connection/without-connection.xml.gz</exclude>
+                        <exclude>src/test/resources/keystore.jks</exclude>
+                        <exclude>src/test/resources/truststore.jks</exclude>
+                    </excludes>
+                </configuration>
+            </plugin>
         </plugins>
     </build>
 
diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/AggregateNiFiInstance.java b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/AggregateNiFiInstance.java
index 688e14a..9358e86 100644
--- a/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/AggregateNiFiInstance.java
+++ b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/AggregateNiFiInstance.java
@@ -30,10 +30,10 @@ public class AggregateNiFiInstance implements NiFiInstance {
     }
 
     @Override
-    public void start() {
+    public void start(boolean waitForCompletion) {
         for (final NiFiInstance instance : instances) {
             if (instance.isAutoStart()) {
-                instance.start();
+                instance.start(waitForCompletion);
             }
         }
     }
@@ -120,6 +120,13 @@ public class AggregateNiFiInstance implements NiFiInstance {
     }
 
     @Override
+    public void setFlowXmlGz(final File flowXmlGz) throws IOException {
+        for (final NiFiInstance instance : instances) {
+            instance.setFlowXmlGz(flowXmlGz);
+        }
+    }
+
+    @Override
     public void setProperties(final Map<String, String> properties) throws IOException {
         for (final NiFiInstance instance : instances) {
             instance.setProperties(properties);
diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/NiFiClientUtil.java b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/NiFiClientUtil.java
index cd268b0..bb7b54b 100644
--- a/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/NiFiClientUtil.java
+++ b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/NiFiClientUtil.java
@@ -32,6 +32,7 @@ import org.apache.nifi.web.api.dto.ControllerServiceDTO;
 import org.apache.nifi.web.api.dto.CounterDTO;
 import org.apache.nifi.web.api.dto.CountersSnapshotDTO;
 import org.apache.nifi.web.api.dto.FlowFileSummaryDTO;
+import org.apache.nifi.web.api.dto.FlowSnippetDTO;
 import org.apache.nifi.web.api.dto.NodeDTO;
 import org.apache.nifi.web.api.dto.ParameterContextDTO;
 import org.apache.nifi.web.api.dto.ParameterContextReferenceDTO;
@@ -397,7 +398,27 @@ public class NiFiClientUtil {
         }
 
         for (final ProcessGroupEntity group : rootFlowDTO.getProcessGroups()) {
-            waitForProcessorsStopped(group.getId());
+            waitForProcessorsStopped(group.getComponent());
+        }
+    }
+
+    private void waitForProcessorsStopped(final ProcessGroupDTO group) throws IOException, NiFiClientException {
+        final FlowSnippetDTO groupContents = group.getContents();
+        if (groupContents == null) {
+            return;
+        }
+
+        for (final ProcessorDTO processor : groupContents.getProcessors()) {
+            try {
+                waitForStoppedProcessor(processor.getId());
+            } catch (InterruptedException e) {
+                Thread.currentThread().interrupt();
+                throw new NiFiClientException("Interrupted while waiting for Processor with ID " + processor.getId() + " to stop");
+            }
+        }
+
+        for (final ProcessGroupDTO child : groupContents.getProcessGroups()) {
+            waitForProcessorsStopped(child);
         }
     }
 
diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/NiFiInstance.java b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/NiFiInstance.java
index d24cb7b..e8b1545 100644
--- a/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/NiFiInstance.java
+++ b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/NiFiInstance.java
@@ -28,10 +28,14 @@ public interface NiFiInstance {
      */
     void createEnvironment() throws IOException;
 
+    default void start() {
+        start(true);
+    }
+
     /**
      * Starts the NiFi instance and waits until the startup is complete
      */
-    void start();
+    void start(boolean waitForCompletion);
 
     /**
      * Shuts down the NiFi instance
@@ -94,6 +98,12 @@ public interface NiFiInstance {
     void setProperty(String propertyName, String propertyValue) throws IOException;
 
     /**
+     * Change the value of the flow that should be loaded on startup
+     * @param flowXmlGz the file that contains the flow that should be loaded on startup
+     */
+    void setFlowXmlGz(final File flowXmlGz) throws IOException;
+
+    /**
      * Change the values of the given properties in nifi.properties. Any property that is not present in the given map will remain unchanged. If the node is already running, this change will not take
      * effect until the instance is stopped and started again.
      *
diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/NiFiSystemIT.java b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/NiFiSystemIT.java
index 23e04ba..04a376b 100644
--- a/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/NiFiSystemIT.java
+++ b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/NiFiSystemIT.java
@@ -158,8 +158,12 @@ public abstract class NiFiSystemIT {
     }
 
     protected NiFiClient createClient() {
+        return createClient(getClientApiPort());
+    }
+
+    protected NiFiClient createClient(final int port) {
         final NiFiClientConfig clientConfig = new NiFiClientConfig.Builder()
-            .baseUrl("http://localhost:" + getClientApiPort())
+            .baseUrl("http://localhost:" + port)
             .connectTimeout(30000)
             .readTimeout(30000)
             .build();
diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/SpawnedStandaloneNiFiInstanceFactory.java b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/SpawnedStandaloneNiFiInstanceFactory.java
index bf5c40e..081eb1e 100644
--- a/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/SpawnedStandaloneNiFiInstanceFactory.java
+++ b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/SpawnedStandaloneNiFiInstanceFactory.java
@@ -79,7 +79,7 @@ public class SpawnedStandaloneNiFiInstanceFactory implements NiFiInstanceFactory
         }
 
         @Override
-        public void start() {
+        public void start(final boolean waitForCompletion) {
             if (runNiFi != null) {
                 throw new IllegalStateException("NiFi has already been started");
             }
@@ -94,7 +94,10 @@ public class SpawnedStandaloneNiFiInstanceFactory implements NiFiInstanceFactory
 
             try {
                 runNiFi.start(false);
-                waitForStartup();
+
+                if (waitForCompletion) {
+                    waitForStartup();
+                }
             } catch (IOException e) {
                 throw new RuntimeException("Failed to start NiFi", e);
             }
@@ -124,6 +127,12 @@ public class SpawnedStandaloneNiFiInstanceFactory implements NiFiInstanceFactory
             // Copy truststore
             final File destinationTruststore = new File(destinationCertsDir, "truststore.jks");
             Files.copy(Paths.get("src/test/resources/truststore.jks"), destinationTruststore.toPath());
+
+            final File flowXmlGz = instanceConfiguration.getFlowXmlGz();
+            if (flowXmlGz != null) {
+                final File destinationFlowXmlGz = new File(destinationConf, "flow.xml.gz");
+                Files.copy(flowXmlGz.toPath(), destinationFlowXmlGz.toPath());
+            }
         }
 
         private void copyContents(final File dir, final File destinationDir) throws IOException {
@@ -148,6 +157,13 @@ public class SpawnedStandaloneNiFiInstanceFactory implements NiFiInstanceFactory
             }
         }
 
+        @Override
+        public void setFlowXmlGz(final File flowXmlGz) throws IOException {
+            final File destinationConf = new File(instanceDirectory, "conf");
+            final File destinationFlowXmlGz = new File(destinationConf, "flow.xml.gz");
+            destinationFlowXmlGz.delete();
+            Files.copy(flowXmlGz.toPath(), destinationFlowXmlGz.toPath());
+        }
 
         private void waitForStartup() throws IOException {
             final NiFiClient client = createClient();
diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/clustering/JoinCluslterWithMissingConnectionWithData.java b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/clustering/JoinCluslterWithMissingConnectionWithData.java
new file mode 100644
index 0000000..640e121
--- /dev/null
+++ b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/clustering/JoinCluslterWithMissingConnectionWithData.java
@@ -0,0 +1,141 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.tests.system.clustering;
+
+import org.apache.nifi.tests.system.InstanceConfiguration;
+import org.apache.nifi.tests.system.NiFiInstance;
+import org.apache.nifi.tests.system.NiFiInstanceFactory;
+import org.apache.nifi.tests.system.NiFiSystemIT;
+import org.apache.nifi.tests.system.SpawnedClusterNiFiInstanceFactory;
+import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientException;
+import org.apache.nifi.web.api.dto.NodeDTO;
+import org.apache.nifi.web.api.dto.flow.FlowDTO;
+import org.apache.nifi.web.api.entity.ConnectionEntity;
+import org.apache.nifi.web.api.entity.NodeEntity;
+import org.apache.nifi.web.api.entity.ProcessorEntity;
+import org.junit.Test;
+
+import java.io.IOException;
+
+public class JoinCluslterWithMissingConnectionWithData  extends NiFiSystemIT {
+    private static final String GENERATE_UUID = "6be9a7e7-016e-1000-0000-00004700499d";
+    private static final String CONNECTION_UUID = "6be9a991-016e-1000-ffff-fffffebf0217";
+
+    @Override
+    protected NiFiInstanceFactory getInstanceFactory() {
+        return new SpawnedClusterNiFiInstanceFactory(
+            new InstanceConfiguration.Builder()
+                .bootstrapConfig("src/test/resources/conf/clustered/node1/bootstrap.conf")
+                .instanceDirectory("target/node1")
+                .flowXml("src/test/resources/flows/missing-connection/with-connection.xml.gz")
+                .build(),
+            new InstanceConfiguration.Builder()
+                .bootstrapConfig("src/test/resources/conf/clustered/node2/bootstrap.conf")
+                .instanceDirectory("target/node2")
+                .flowXml("src/test/resources/flows/missing-connection/with-connection.xml.gz")
+                .build()
+        );
+    }
+
+    @Override
+    protected boolean isDestroyFlowAfterEachTest() {
+        // Do not destroy the flow because there is only a single test in the class and because the expected state
+        // is for Node 2 to be a part of the cluster but disconnected.
+        return false;
+    }
+
+    @Test
+    public void testFailsToJoinWithMissingConnectionThatHasData() throws NiFiClientException, IOException, InterruptedException {
+        // Create the flow
+        ProcessorEntity generate = getNifiClient().getProcessorClient().getProcessor(GENERATE_UUID);
+
+        // Start Generate Processor
+        generate = getNifiClient().getProcessorClient().startProcessor(generate);
+
+        // Wait for data to be queued up, one FlowFile for each node.
+        waitFor(this::isDataQueued);
+
+        // Stop the processor
+        getNifiClient().getProcessorClient().stopProcessor(generate);
+        getClientUtil().waitForStoppedProcessor(generate.getId());
+
+        // Disconnect and remove node 2 from the cluster. Then stop the node.
+        final NodeDTO node2Dto = getNifiClient().getControllerClient().getNodes().getCluster().getNodes().stream()
+            .filter(nodeDto -> nodeDto.getApiPort() == 5672)
+            .findAny()
+            .get();
+
+        final NodeEntity nodeEntity = new NodeEntity();
+        nodeEntity.setNode(node2Dto);
+        node2Dto.setStatus("DISCONNECTING");
+        getNifiClient().getControllerClient().disconnectNode(node2Dto.getNodeId(), nodeEntity);
+        waitFor(() -> isNodeDisconnected(5672));
+
+        final NiFiInstance node2 = getNiFiInstance().getNodeInstance(2);
+        node2.stop();
+
+        // Remove node from the cluster
+        getNifiClient().getControllerClient().deleteNode(node2Dto.getNodeId());
+        waitFor(() -> isNodeRemoved(5672));
+
+        // Drop the data in the queue and delete the queue.
+        getClientUtil().emptyQueue(CONNECTION_UUID);
+
+        getNifiClient().getConnectionClient().deleteConnection(CONNECTION_UUID, "test-client-id", 0);
+
+        node2.start(false);
+
+        // Node should fail to connect but instead should be disconnected.
+        waitFor(() -> isNodeDisconnected(5672));
+    }
+
+    private boolean isNodeRemoved(final int apiPort) {
+        try {
+            return getNifiClient().getControllerClient().getNodes().getCluster().getNodes().stream()
+                .noneMatch(dto -> dto.getApiPort() == apiPort);
+        } catch (Exception e) {
+            return false;
+        }
+    }
+
+    private boolean isNodeDisconnected(final int apiPort) {
+        try {
+            final NodeDTO nodeDto = getNifiClient().getControllerClient().getNodes().getCluster().getNodes().stream()
+                .filter(dto -> dto.getApiPort() == apiPort)
+                .findAny()
+                .get();
+
+            return "DISCONNECTED".equals(nodeDto.getStatus());
+        } catch (Exception e) {
+            return false;
+        }
+    }
+
+    private boolean isDataQueued() {
+        final FlowDTO flowDto;
+        try {
+            flowDto = getNifiClient().getFlowClient().getProcessGroup("root").getProcessGroupFlow().getFlow();
+        } catch (Exception e) {
+            e.printStackTrace();
+            return false;
+        }
+
+        final ConnectionEntity connectionEntity = flowDto.getConnections().iterator().next();
+        final Integer queuedCount = connectionEntity.getStatus().getAggregateSnapshot().getFlowFilesQueued();
+        return queuedCount == 2;
+    }
+}
\ No newline at end of file
diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/clustering/JoinClusterWithDifferentFlow.java b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/clustering/JoinClusterWithDifferentFlow.java
new file mode 100644
index 0000000..580ceb1
--- /dev/null
+++ b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/clustering/JoinClusterWithDifferentFlow.java
@@ -0,0 +1,253 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.tests.system.clustering;
+
+import org.apache.nifi.controller.serialization.FlowEncodingVersion;
+import org.apache.nifi.controller.serialization.FlowFromDOMFactory;
+import org.apache.nifi.encrypt.StringEncryptor;
+import org.apache.nifi.tests.system.InstanceConfiguration;
+import org.apache.nifi.tests.system.NiFiInstance;
+import org.apache.nifi.tests.system.NiFiInstanceFactory;
+import org.apache.nifi.tests.system.NiFiSystemIT;
+import org.apache.nifi.tests.system.SpawnedClusterNiFiInstanceFactory;
+import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClient;
+import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientException;
+import org.apache.nifi.util.NiFiProperties;
+import org.apache.nifi.web.api.dto.AffectedComponentDTO;
+import org.apache.nifi.web.api.dto.FlowSnippetDTO;
+import org.apache.nifi.web.api.dto.NodeDTO;
+import org.apache.nifi.web.api.dto.ParameterContextDTO;
+import org.apache.nifi.web.api.dto.ParameterContextReferenceDTO;
+import org.apache.nifi.web.api.dto.ProcessGroupDTO;
+import org.apache.nifi.web.api.dto.ProcessorDTO;
+import org.apache.nifi.web.api.dto.flow.FlowDTO;
+import org.apache.nifi.web.api.dto.flow.ProcessGroupFlowDTO;
+import org.apache.nifi.web.api.entity.AffectedComponentEntity;
+import org.apache.nifi.web.api.entity.ClusterEntity;
+import org.apache.nifi.web.api.entity.ControllerServiceEntity;
+import org.apache.nifi.web.api.entity.ControllerServicesEntity;
+import org.apache.nifi.web.api.entity.NodeEntity;
+import org.apache.nifi.web.api.entity.ParameterEntity;
+import org.apache.nifi.web.api.entity.ProcessorEntity;
+import org.junit.Test;
+import org.w3c.dom.Document;
+import org.w3c.dom.Element;
+import org.xml.sax.InputSource;
+import org.xml.sax.SAXException;
+
+import javax.xml.parsers.DocumentBuilder;
+import javax.xml.parsers.DocumentBuilderFactory;
+import javax.xml.parsers.ParserConfigurationException;
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.StringReader;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.zip.GZIPInputStream;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
+public class JoinClusterWithDifferentFlow extends NiFiSystemIT {
+    @Override
+    protected NiFiInstanceFactory getInstanceFactory() {
+        return new SpawnedClusterNiFiInstanceFactory(
+            new InstanceConfiguration.Builder()
+                .bootstrapConfig("src/test/resources/conf/clustered/node1/bootstrap.conf")
+                .instanceDirectory("target/node1")
+                .flowXml(new File("src/test/resources/flows/mismatched-flows/flow1.xml.gz"))
+                .build(),
+            new InstanceConfiguration.Builder()
+                .bootstrapConfig("src/test/resources/conf/clustered/node2/bootstrap.conf")
+                .instanceDirectory("target/node2")
+                .flowXml(new File("src/test/resources/flows/mismatched-flows/flow2.xml.gz"))
+                .build()
+        );
+    }
+
+
+    @Test
+    public void testStartupWithDifferentFlow() throws IOException, SAXException, ParserConfigurationException, NiFiClientException, InterruptedException {
+        final NiFiInstance node2 = getNiFiInstance().getNodeInstance(2);
+        final File node2ConfDir = new File(node2.getInstanceDirectory(), "conf");
+
+        final File backupFile = getBackupFile(node2ConfDir);
+        final NodeDTO node2Dto = getNodeDTO(5672);
+
+        verifyFlowContentsOnDisk(backupFile);
+        disconnectNode(node2Dto);
+        verifyInMemoryFlowContents();
+
+        // Reconnect the node so that we can properly shutdown
+        reconnectNode(node2Dto);
+    }
+
+
+    private File getBackupFile(final File confDir) {
+        final File[] flowXmlFileArray = confDir.listFiles(file -> file.getName().startsWith("flow") && file.getName().endsWith(".xml.gz"));
+        final List<File> flowXmlFiles = new ArrayList<>(Arrays.asList(flowXmlFileArray));
+        assertEquals(2, flowXmlFiles.size());
+        flowXmlFiles.removeIf(file -> file.getName().equals("flow.xml.gz"));
+
+        assertEquals(1, flowXmlFiles.size());
+        final File backupFile = flowXmlFiles.get(0);
+        return backupFile;
+    }
+
+    private void verifyFlowContentsOnDisk(final File backupFile) throws IOException, SAXException, ParserConfigurationException {
+        // Read the flow and make sure that the backup looks the same as the original. We don't just do a byte comparison because the compression may result in different
+        // gzipped bytes and because if the two flows do differ, we want to have the String representation so that we can compare to see how they are different.
+        final String flowXml = readFlow(backupFile);
+        final String expectedFlow = readFlow(new File("src/test/resources/flows/mismatched-flows/flow2.xml.gz"));
+
+        assertEquals(expectedFlow, flowXml);
+
+        // Verify some of the values that were persisted to disk
+        final File confDir = backupFile.getParentFile();
+        final String loadedFlow = readFlow(new File(confDir, "flow.xml.gz"));
+
+        final DocumentBuilder documentBuilder = DocumentBuilderFactory.newInstance().newDocumentBuilder();
+        final Document document = documentBuilder.parse(new InputSource(new StringReader(loadedFlow)));
+        final Element rootElement = (Element) document.getElementsByTagName("flowController").item(0);
+        final FlowEncodingVersion encodingVersion = FlowEncodingVersion.parse(rootElement);
+
+        final NiFiInstance node2 = getNiFiInstance().getNodeInstance(2);
+        final StringEncryptor encryptor = createEncryptorFromProperties(node2.getProperties());
+        final Element rootGroupElement = (Element) rootElement.getElementsByTagName("rootGroup").item(0);
+
+        final ProcessGroupDTO groupDto = FlowFromDOMFactory.getProcessGroup(null, rootGroupElement, encryptor, encodingVersion);
+        final Set<ProcessGroupDTO> childGroupDtos = groupDto.getContents().getProcessGroups();
+        assertEquals(1, childGroupDtos.size());
+
+        final ProcessGroupDTO childGroup = childGroupDtos.iterator().next();
+        assertFalse(childGroup.getId().endsWith("00"));
+        final FlowSnippetDTO childContents = childGroup.getContents();
+
+        final Set<ProcessorDTO> childProcessors = childContents.getProcessors();
+        assertEquals(1, childProcessors.size());
+
+        final ProcessorDTO procDto = childProcessors.iterator().next();
+        assertFalse(procDto.getId().endsWith("00"));
+        assertFalse(procDto.getName().endsWith("00"));
+    }
+
+
+    private NodeDTO getNodeDTO(final int apiPort) throws NiFiClientException, IOException {
+        final ClusterEntity clusterEntity = getNifiClient().getControllerClient().getNodes();
+        final NodeDTO node2Dto = clusterEntity.getCluster().getNodes().stream()
+            .filter(nodeDto -> nodeDto.getApiPort() == apiPort)
+            .findAny()
+            .orElseThrow(() -> new RuntimeException("Could not locate Node 2"));
+
+        return node2Dto;
+    }
+
+    private void disconnectNode(final NodeDTO nodeDto) throws NiFiClientException, IOException, InterruptedException {
+        // Disconnect Node 2 so that we can go to the node directly via the REST API and ensure that the flow is correct.
+        final NodeEntity nodeEntity = new NodeEntity();
+        nodeEntity.setNode(nodeDto);
+
+        getNifiClient().getControllerClient().disconnectNode(nodeDto.getNodeId(), nodeEntity);
+
+        // Give the node a second to disconnect
+        Thread.sleep(1000L);
+    }
+
+    private void reconnectNode(final NodeDTO nodeDto) throws NiFiClientException, IOException {
+        final NodeEntity nodeEntity = new NodeEntity();
+        nodeEntity.setNode(nodeDto);
+        getNifiClient().getControllerClient().connectNode(nodeDto.getNodeId(), nodeEntity);
+        waitForAllNodesConnected();
+    }
+
+    private void verifyInMemoryFlowContents() throws NiFiClientException, IOException, InterruptedException {
+        final NiFiClient node2Client = createClient(5672);
+        final ProcessGroupFlowDTO rootGroupFlow = node2Client.getFlowClient().getProcessGroup("root").getProcessGroupFlow();
+        final FlowDTO flowDto = rootGroupFlow.getFlow();
+        assertEquals(1, flowDto.getProcessGroups().size());
+
+        final ParameterContextReferenceDTO paramContextReference = flowDto.getProcessGroups().iterator().next().getParameterContext().getComponent();
+        assertEquals("65b6403c-016e-1000-900b-357b13fcc7c4", paramContextReference.getId());
+        assertEquals("Context 1", paramContextReference.getName());
+
+        ProcessorEntity generateFlowFileEntity = node2Client.getProcessorClient().getProcessor("65b8f293-016e-1000-7b8f-6c6752fa921b");
+        final Map<String, String> generateProperties = generateFlowFileEntity.getComponent().getConfig().getProperties();
+        assertEquals("01 B", generateProperties.get("File Size"));
+        assertEquals("1", generateProperties.get("Batch Size"));
+
+        assertEquals("1 hour", generateFlowFileEntity.getComponent().getConfig().getSchedulingPeriod());
+
+        String currentState = null;
+        while ("RUNNING".equals(currentState)) {
+            Thread.sleep(50L);
+            generateFlowFileEntity = node2Client.getProcessorClient().getProcessor("65b8f293-016e-1000-7b8f-6c6752fa921b");
+            currentState = generateFlowFileEntity.getComponent().getState();
+        }
+
+        final ParameterContextDTO contextDto = node2Client.getParamContextClient().getParamContext(paramContextReference.getId()).getComponent();
+        assertEquals(2, contextDto.getBoundProcessGroups().size());
+        assertEquals(1, contextDto.getParameters().size());
+        final ParameterEntity parameterEntity = contextDto.getParameters().iterator().next();
+        assertEquals("ABC", parameterEntity.getParameter().getName());
+        assertEquals("XYZ", parameterEntity.getParameter().getValue());
+
+        final Set<AffectedComponentEntity> affectedComponentEntities = parameterEntity.getParameter().getReferencingComponents();
+        assertEquals(1, affectedComponentEntities.size());
+        final AffectedComponentDTO affectedComponent = affectedComponentEntities.iterator().next().getComponent();
+        assertEquals("65b8f293-016e-1000-7b8f-6c6752fa921b", affectedComponent.getId());
+        assertEquals(AffectedComponentDTO.COMPONENT_TYPE_PROCESSOR, affectedComponent.getReferenceType());
+
+        // The original Controller Service, whose UUID ended with 00 should be removed and a new one inherited.
+        final ControllerServicesEntity controllerLevelServices = node2Client.getFlowClient().getControllerServices();
+        assertEquals(1, controllerLevelServices.getControllerServices().size());
+
+        final ControllerServiceEntity firstService = controllerLevelServices.getControllerServices().iterator().next();
+        assertFalse(firstService.getId().endsWith("00"));
+    }
+
+    private StringEncryptor createEncryptorFromProperties(Properties nifiProperties) {
+        final String algorithm = nifiProperties.getProperty(NiFiProperties.SENSITIVE_PROPS_ALGORITHM);
+        final String provider = nifiProperties.getProperty(NiFiProperties.SENSITIVE_PROPS_PROVIDER);
+        final String password = nifiProperties.getProperty(NiFiProperties.SENSITIVE_PROPS_KEY);
+        return StringEncryptor.createEncryptor(algorithm, provider, password);
+    }
+
+    private String readFlow(final File file) throws IOException {
+        final ByteArrayOutputStream baos = new ByteArrayOutputStream();
+
+        try (final InputStream fis = new FileInputStream(file);
+             final InputStream gzipIn = new GZIPInputStream(fis)) {
+
+            final byte[] buffer = new byte[4096];
+            int len;
+            while ((len = gzipIn.read(buffer)) > 0) {
+                baos.write(buffer, 0, len);
+            }
+        }
+
+        final byte[] bytes = baos.toByteArray();
+        return new String(bytes, StandardCharsets.UTF_8);
+    }
+}
\ No newline at end of file
diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/clustering/JoinClusterWithMissingConnectionNoData.java b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/clustering/JoinClusterWithMissingConnectionNoData.java
new file mode 100644
index 0000000..37d681e
--- /dev/null
+++ b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/clustering/JoinClusterWithMissingConnectionNoData.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.tests.system.clustering;
+
+import org.apache.nifi.tests.system.InstanceConfiguration;
+import org.apache.nifi.tests.system.NiFiInstanceFactory;
+import org.apache.nifi.tests.system.NiFiSystemIT;
+import org.apache.nifi.tests.system.SpawnedClusterNiFiInstanceFactory;
+import org.junit.Test;
+
+public class JoinClusterWithMissingConnectionNoData extends NiFiSystemIT {
+    @Override
+    protected NiFiInstanceFactory getInstanceFactory() {
+        return new SpawnedClusterNiFiInstanceFactory(
+            new InstanceConfiguration.Builder()
+                .bootstrapConfig("src/test/resources/conf/clustered/node1/bootstrap.conf")
+                .instanceDirectory("target/node1")
+                .flowXml("src/test/resources/flows/missing-connection/with-connection.xml.gz")
+                .build(),
+            new InstanceConfiguration.Builder()
+                .bootstrapConfig("src/test/resources/conf/clustered/node2/bootstrap.conf")
+                .instanceDirectory("target/node2")
+                .flowXml("src/test/resources/flows/missing-connection/without-connection.xml.gz")
+                .build()
+        );
+    }
+
+    @Test
+    public void testSuccessfulJoinWithMissingConnectionNoData() {
+        // If the node joins then the test is successful.
+        waitForAllNodesConnected();
+    }
+
+}
diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/resources/conf/clustered/node2/bootstrap.conf b/nifi-system-tests/nifi-system-test-suite/src/test/resources/conf/clustered/node2/bootstrap.conf
index b348aa0..0f4b8f3 100644
--- a/nifi-system-tests/nifi-system-test-suite/src/test/resources/conf/clustered/node2/bootstrap.conf
+++ b/nifi-system-tests/nifi-system-test-suite/src/test/resources/conf/clustered/node2/bootstrap.conf
@@ -28,6 +28,6 @@ java.arg.3=-Xmx128m
 
 java.arg.14=-Djava.awt.headless=true
 
-#java.arg.debug=-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=8003
+java.arg.debug=-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=8003
 
 nifi.bootstrap.sensitive.key=
\ No newline at end of file
diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/resources/conf/clustered/node2/logback.xml b/nifi-system-tests/nifi-system-test-suite/src/test/resources/conf/clustered/node2/logback.xml
index 0a91790..b476a5b 100644
--- a/nifi-system-tests/nifi-system-test-suite/src/test/resources/conf/clustered/node2/logback.xml
+++ b/nifi-system-tests/nifi-system-test-suite/src/test/resources/conf/clustered/node2/logback.xml
@@ -88,6 +88,8 @@
     <logger name="org.apache.nifi.processors.standard.LogMessage" level="INFO"/>
     <logger name="org.apache.nifi.controller.repository.StandardProcessSession" level="WARN" />
 
+    <logger name="org.apache.nifi.controller.StandardFlowSynchronizer" level="DEBUG" />
+    <logger name="org.apache.nifi.controller.inheritance" level="DEBUG" />
 
     <logger name="org.apache.zookeeper.ClientCnxn" level="ERROR" />
     <logger name="org.apache.zookeeper.server.NIOServerCnxn" level="ERROR" />
diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/resources/conf/default/logback.xml b/nifi-system-tests/nifi-system-test-suite/src/test/resources/conf/default/logback.xml
index 73c5cdc..0580768 100644
--- a/nifi-system-tests/nifi-system-test-suite/src/test/resources/conf/default/logback.xml
+++ b/nifi-system-tests/nifi-system-test-suite/src/test/resources/conf/default/logback.xml
@@ -89,7 +89,6 @@
     <logger name="org.apache.nifi.processors.standard.LogMessage" level="INFO"/>
     <logger name="org.apache.nifi.controller.repository.StandardProcessSession" level="WARN" />
 
-
     <logger name="org.apache.zookeeper.ClientCnxn" level="ERROR" />
     <logger name="org.apache.zookeeper.server.NIOServerCnxn" level="ERROR" />
     <logger name="org.apache.zookeeper.server.NIOServerCnxnFactory" level="ERROR" />
diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/resources/flows/mismatched-flows/flow1.xml.gz b/nifi-system-tests/nifi-system-test-suite/src/test/resources/flows/mismatched-flows/flow1.xml.gz
new file mode 100644
index 0000000..991645f
Binary files /dev/null and b/nifi-system-tests/nifi-system-test-suite/src/test/resources/flows/mismatched-flows/flow1.xml.gz differ
diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/resources/flows/mismatched-flows/flow2.xml.gz b/nifi-system-tests/nifi-system-test-suite/src/test/resources/flows/mismatched-flows/flow2.xml.gz
new file mode 100644
index 0000000..b17e57d
Binary files /dev/null and b/nifi-system-tests/nifi-system-test-suite/src/test/resources/flows/mismatched-flows/flow2.xml.gz differ
diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/resources/flows/missing-connection/with-connection.xml.gz b/nifi-system-tests/nifi-system-test-suite/src/test/resources/flows/missing-connection/with-connection.xml.gz
new file mode 100644
index 0000000..9b07cfc
Binary files /dev/null and b/nifi-system-tests/nifi-system-test-suite/src/test/resources/flows/missing-connection/with-connection.xml.gz differ
diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/resources/flows/missing-connection/without-connection.xml.gz b/nifi-system-tests/nifi-system-test-suite/src/test/resources/flows/missing-connection/without-connection.xml.gz
new file mode 100644
index 0000000..b5615d7
Binary files /dev/null and b/nifi-system-tests/nifi-system-test-suite/src/test/resources/flows/missing-connection/without-connection.xml.gz differ