You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by bb...@apache.org on 2019/04/10 20:47:27 UTC

[nifi] branch master updated: NIFI-6183: Implemented base class for Framework-level integration tests that is responsible for instantiating the key components for integration tests, including FlowController, WriteAheadFlowFileRepository, WriteAheadProvenanceRepository, FileSystemRepository, FileSystemSwapManager, StandardFlowFileQueue, StandardProcessScheduler, etc. Implemented several integration tests to span different parts of the framework

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 91b2283  NIFI-6183: Implemented base class for Framework-level integration tests that is responsible for instantiating the key components for integration tests, including FlowController, WriteAheadFlowFileRepository, WriteAheadProvenanceRepository, FileSystemRepository, FileSystemSwapManager, StandardFlowFileQueue, StandardProcessScheduler, etc. Implemented several integration tests to span different parts of the framework
91b2283 is described below

commit 91b2283f49262b7d4c8a2fecbc757a360b0f8321
Author: Mark Payne <ma...@hotmail.com>
AuthorDate: Fri Mar 29 09:22:39 2019 -0400

    NIFI-6183: Implemented base class for Framework-level integration tests that is responsible for instantiating the key components for integration tests, including FlowController, WriteAheadFlowFileRepository, WriteAheadProvenanceRepository, FileSystemRepository, FileSystemSwapManager, StandardFlowFileQueue, StandardProcessScheduler, etc. Implemented several integration tests to span different parts of the framework
    
    This closes #3423.
    
    Signed-off-by: Bryan Bende <bb...@apache.org>
---
 .../repository/RepositoryRecordSerdeFactory.java   |  75 +---
 ...a => StandardRepositoryRecordSerdeFactory.java} |  12 +-
 .../nifi/controller/AbstractComponentNode.java     |  49 +--
 .../org/apache/nifi/events/BulletinFactory.java    |   6 +-
 .../nifi-framework/nifi-framework-core/pom.xml     |   6 +
 .../org/apache/nifi/controller/FlowController.java |  24 +-
 .../controller/queue/ConnectionEventListener.java  |  10 +
 .../repository/FileSystemRepository.java           |   2 +-
 .../controller/repository/RepositoryContext.java   |  22 +-
 .../repository/StandardProcessSession.java         |  33 +-
 .../repository/StandardQueueProvider.java          |  10 +-
 .../repository/WriteAheadFlowFileRepository.java   |   6 +-
 .../metrics/RingBufferEventRepository.java         |   1 -
 .../metrics/SecondPrecisionEventContainer.java     |   4 +-
 .../scheduling/TimerDrivenSchedulingAgent.java     |  12 +-
 .../nifi/controller/tasks/ConnectableTask.java     |  15 +-
 .../apache/nifi/groups/StandardProcessGroup.java   |  10 +-
 .../repository/TestStandardProcessSession.java     |   8 +-
 .../TestWriteAheadFlowFileRepository.java          |   2 +-
 .../integration/AlwaysAuthorizedAuthorizer.java    |  56 +++
 .../DirectInjectionExtensionManager.java           |  64 +++
 .../nifi/integration/FrameworkIntegrationTest.java | 461 +++++++++++++++++++++
 .../apache/nifi/integration/NopAuditService.java   |  60 +++
 .../cs/ControllerServiceReferenceIT.java           |  70 ++++
 .../cs/ControllerServiceReferencingProcessor.java  |  52 +++
 .../org/apache/nifi/integration/cs/Counter.java}   |   9 +-
 .../integration/cs/CounterControllerService.java}  |  21 +-
 .../cs/LongValidatingControllerService.java        |  59 +++
 .../nifi/integration/cs/NopControllerService.java} |   8 +-
 .../cs/NopServiceReferencingProcessor.java         |  46 ++
 .../flowfilerepo/OOMEFlowFileRepoUpdateIT.java     |  76 ++++
 .../OOMEWriteAheadFlowFileRepository.java          | 142 +++++++
 .../integration/lifecycle/ContentCleanupIT.java    | 184 ++++++++
 .../lifecycle/FlowFileRepositoryLifecycleIT.java   | 147 +++++++
 .../processgroup/StandardProcessGroupIT.java       | 100 +++++
 .../integration/processor/BiConsumerProcessor.java |  57 +++
 .../integration/provenance/ProvenanceEventsIT.java | 423 +++++++++++++++++++
 .../nifi/integration/versioned/ImportFlowIT.java   | 163 ++++++++
 .../resources/int-tests/default-nifi.properties    | 256 ++++++++++++
 .../test/resources/int-tests/state-management.xml  |  24 ++
 .../src/test/resources/logback-test.xml            |   1 +
 .../nar/StandardExtensionDiscoveringManager.java   |  76 ++--
 .../repository/claim/StandardContentClaim.java     |   4 +
 .../apache/nifi/remote/HttpRemoteSiteListener.java |  19 +-
 .../org/apache/nifi/remote/RemoteSiteListener.java |   6 +-
 .../nifi/remote/SocketRemoteSiteListener.java      |   6 +-
 .../nifi/web/dao/impl/StandardProcessGroupDAO.java |   2 +-
 .../provenance/index/lucene/EventIndexTask.java    |  40 +-
 .../provenance/index/lucene/LuceneEventIndex.java  |  54 ++-
 .../nifi/provenance/lucene/SimpleIndexManager.java |  38 +-
 50 files changed, 2756 insertions(+), 275 deletions(-)

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 c19fa94..38b7845 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
@@ -14,82 +14,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 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.SerDeFactory;
-import org.wali.UpdateType;
-
-public class RepositoryRecordSerdeFactory implements SerDeFactory<RepositoryRecord> {
-    private final String LEGACY_SERDE_ENCODING_NAME = "org.apache.nifi.controller.repository.WriteAheadFlowFileRepository$WriteAheadRecordSerde";
-    private final ResourceClaimManager resourceClaimManager;
-    private Map<String, FlowFileQueue> flowFileQueueMap = null;
-
-    public RepositoryRecordSerdeFactory(final ResourceClaimManager claimManager) {
-        this.resourceClaimManager = claimManager;
-    }
-
-    protected 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) {
-        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();
-    }
-
-    @Override
-    public UpdateType getUpdateType(final RepositoryRecord record) {
-        switch (record.getType()) {
-            case CONTENTMISSING:
-            case DELETE:
-                return UpdateType.DELETE;
-            case CREATE:
-                return UpdateType.CREATE;
-            case UPDATE:
-                return UpdateType.UPDATE;
-            case SWAP_OUT:
-                return UpdateType.SWAP_OUT;
-            case SWAP_IN:
-                return UpdateType.SWAP_IN;
-        }
-        return null;
-    }
+import java.util.Map;
 
-    @Override
-    public String getLocation(final RepositoryRecord record) {
-        return record.getSwapLocation();
-    }
+public interface RepositoryRecordSerdeFactory extends SerDeFactory<RepositoryRecord> {
+    void setQueueMap(Map<String, FlowFileQueue> queueMap);
 
+    Long getRecordIdentifier(RepositoryRecord record);
 }
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/StandardRepositoryRecordSerdeFactory.java
similarity index 92%
copy from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/RepositoryRecordSerdeFactory.java
copy to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/StandardRepositoryRecordSerdeFactory.java
index c19fa94..f24ac08 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/StandardRepositoryRecordSerdeFactory.java
@@ -17,24 +17,24 @@
 
 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.SerDeFactory;
 import org.wali.UpdateType;
 
-public class RepositoryRecordSerdeFactory implements SerDeFactory<RepositoryRecord> {
+import java.util.Map;
+
+public class StandardRepositoryRecordSerdeFactory implements RepositoryRecordSerdeFactory {
     private final String LEGACY_SERDE_ENCODING_NAME = "org.apache.nifi.controller.repository.WriteAheadFlowFileRepository$WriteAheadRecordSerde";
     private final ResourceClaimManager resourceClaimManager;
     private Map<String, FlowFileQueue> flowFileQueueMap = null;
 
-    public RepositoryRecordSerdeFactory(final ResourceClaimManager claimManager) {
+    public StandardRepositoryRecordSerdeFactory(final ResourceClaimManager claimManager) {
         this.resourceClaimManager = claimManager;
     }
 
-    protected void setQueueMap(final Map<String, FlowFileQueue> queueMap) {
+    @Override
+    public void setQueueMap(final Map<String, FlowFileQueue> queueMap) {
         this.flowFileQueueMap = queueMap;
     }
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractComponentNode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractComponentNode.java
index f78c9d2..d2032a2 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractComponentNode.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractComponentNode.java
@@ -67,7 +67,6 @@ public abstract class AbstractComponentNode implements ComponentNode {
     private final ControllerServiceProvider serviceProvider;
     private final AtomicReference<String> name;
     private final AtomicReference<String> annotationData = new AtomicReference<>();
-    private final AtomicReference<ValidationContext> validationContext = new AtomicReference<>();
     private final String componentType;
     private final String componentCanonicalClass;
     private final ComponentVariableRegistry variableRegistry;
@@ -79,10 +78,13 @@ public abstract class AbstractComponentNode implements ComponentNode {
     private final Lock lock = new ReentrantLock();
     private final ConcurrentMap<PropertyDescriptor, String> properties = new ConcurrentHashMap<>();
     private volatile String additionalResourcesFingerprint;
-    private AtomicReference<ValidationState> validationState = new AtomicReference<>(new ValidationState(ValidationStatus.VALIDATING, Collections.emptyList()));
+    private final AtomicReference<ValidationState> validationState = new AtomicReference<>(new ValidationState(ValidationStatus.VALIDATING, Collections.emptyList()));
     private final ValidationTrigger validationTrigger;
     private volatile boolean triggerValidation = true;
 
+    // guaraded by lock
+    private ValidationContext validationContext = null;
+
     public AbstractComponentNode(final String id,
                                  final ValidationContextFactory validationContextFactory, final ControllerServiceProvider serviceProvider,
                                  final String componentType, final String componentCanonicalClass, final ComponentVariableRegistry variableRegistry,
@@ -575,7 +577,7 @@ public abstract class AbstractComponentNode implements ComponentNode {
     }
 
 
-    private final void onPropertyModified(final PropertyDescriptor descriptor, final String oldValue, final String newValue) {
+    private void onPropertyModified(final PropertyDescriptor descriptor, final String oldValue, final String newValue) {
         try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(extensionManager, getComponent().getClass(), getComponent().getIdentifier())) {
             getComponent().onPropertyModified(descriptor, oldValue, newValue);
         }
@@ -627,13 +629,18 @@ public abstract class AbstractComponentNode implements ComponentNode {
     }
 
     protected void resetValidationState() {
-        validationContext.set(null);
-        validationState.set(new ValidationState(ValidationStatus.VALIDATING, Collections.emptyList()));
+        lock.lock();
+        try {
+            validationContext = null;
+            validationState.set(new ValidationState(ValidationStatus.VALIDATING, Collections.emptyList()));
 
-        if (isTriggerValidation()) {
-            validationTrigger.triggerAsync(this);
-        } else {
-            logger.debug("Reset validation state of {} but will not trigger async validation because trigger has been paused", this);
+            if (isTriggerValidation()) {
+                validationTrigger.triggerAsync(this);
+            } else {
+                logger.debug("Reset validation state of {} but will not trigger async validation because trigger has been paused", this);
+            }
+        } finally {
+            lock.unlock();
         }
     }
 
@@ -714,27 +721,21 @@ public abstract class AbstractComponentNode implements ComponentNode {
     }
 
     protected ValidationContext getValidationContext() {
-        while (true) {
-            ValidationContext context = this.validationContext.get();
+        lock.lock();
+        try {
+            ValidationContext context = this.validationContext;
             if (context != null) {
                 return context;
             }
 
-            // Use a lock here because we want to prevent calls to getProperties() from happening while setProperties() is also happening.
-            final Map<PropertyDescriptor, String> properties;
-            lock.lock();
-            try {
-                properties = getProperties();
-            } finally {
-                lock.unlock();
-            }
+            final Map<PropertyDescriptor, String> properties = getProperties();
             context = getValidationContextFactory().newValidationContext(properties, getAnnotationData(), getProcessGroupIdentifier(), getIdentifier());
 
-            final boolean updated = validationContext.compareAndSet(null, context);
-            if (updated) {
-                logger.debug("Updating validation context to {}", context);
-                return context;
-            }
+            this.validationContext = context;
+            logger.debug("Updating validation context to {}", context);
+            return context;
+        } finally {
+            lock.unlock();
         }
     }
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/events/BulletinFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/events/BulletinFactory.java
index eb00402..76728ba 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/events/BulletinFactory.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/events/BulletinFactory.java
@@ -16,13 +16,13 @@
  */
 package org.apache.nifi.events;
 
-import java.util.concurrent.atomic.AtomicLong;
-
 import org.apache.nifi.connectable.Connectable;
 import org.apache.nifi.groups.ProcessGroup;
 import org.apache.nifi.reporting.Bulletin;
 import org.apache.nifi.reporting.ComponentType;
 
+import java.util.concurrent.atomic.AtomicLong;
+
 public final class BulletinFactory {
 
     private static final AtomicLong currentId = new AtomicLong(0);
@@ -47,7 +47,7 @@ public final class BulletinFactory {
         }
 
         final ProcessGroup group = connectable.getProcessGroup();
-        final String groupId = group == null ? null : group.getIdentifier();
+        final String groupId = connectable.getProcessGroupIdentifier();
         final String groupName = group == null ? null : group.getName();
         return BulletinFactory.createBulletin(groupId, groupName, connectable.getIdentifier(), type, connectable.getName(), category, severity, message);
     }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/pom.xml
index 85c5281..ca5ab87 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/pom.xml
@@ -225,6 +225,12 @@
             <version>1.10.0-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-persistent-provenance-repository</artifactId>
+            <version>1.10.0-SNAPSHOT</version>
+            <scope>test</scope>
+        </dependency>
     </dependencies>
     <build>
         <plugins>
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 b731610..4141832 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
@@ -86,6 +86,7 @@ import org.apache.nifi.controller.repository.FlowFileEventRepository;
 import org.apache.nifi.controller.repository.FlowFileRecord;
 import org.apache.nifi.controller.repository.FlowFileRepository;
 import org.apache.nifi.controller.repository.FlowFileSwapManager;
+import org.apache.nifi.controller.repository.QueueProvider;
 import org.apache.nifi.controller.repository.StandardCounterRepository;
 import org.apache.nifi.controller.repository.StandardFlowFileRecord;
 import org.apache.nifi.controller.repository.StandardQueueProvider;
@@ -294,6 +295,7 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
     private final ProvenanceAuthorizableFactory provenanceAuthorizableFactory;
     private final UserAwareEventAccess eventAccess;
     private final StandardFlowManager flowManager;
+    private final RepositoryContextFactory repositoryContextFactory;
 
     /**
      * true if controller is configured to operate in a clustered environment
@@ -433,7 +435,7 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
             final ExtensionManager extensionManager) {
 
         maxTimerDrivenThreads = new AtomicInteger(10);
-        maxEventDrivenThreads = new AtomicInteger(5);
+        maxEventDrivenThreads = new AtomicInteger(1);
 
         this.encryptor = encryptor;
         this.nifiProperties = nifiProperties;
@@ -484,18 +486,17 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
         processScheduler = new StandardProcessScheduler(timerDrivenEngineRef.get(), this, encryptor, stateManagerProvider, this.nifiProperties);
         eventDrivenWorkerQueue = new EventDrivenWorkerQueue(false, false, processScheduler);
 
-        final RepositoryContextFactory contextFactory = new RepositoryContextFactory(contentRepository, flowFileRepository, flowFileEventRepository, counterRepositoryRef.get(), provenanceRepository);
-
+        repositoryContextFactory = new RepositoryContextFactory(contentRepository, flowFileRepository, flowFileEventRepository, counterRepositoryRef.get(), provenanceRepository);
         this.flowManager = new StandardFlowManager(nifiProperties, sslContext, this, flowFileEventRepository);
 
         controllerServiceProvider = new StandardControllerServiceProvider(this, processScheduler, bulletinRepository);
 
         eventDrivenSchedulingAgent = new EventDrivenSchedulingAgent(
-            eventDrivenEngineRef.get(), controllerServiceProvider, stateManagerProvider, eventDrivenWorkerQueue, contextFactory, maxEventDrivenThreads.get(), encryptor, extensionManager);
+            eventDrivenEngineRef.get(), controllerServiceProvider, stateManagerProvider, eventDrivenWorkerQueue, repositoryContextFactory, maxEventDrivenThreads.get(), encryptor, extensionManager);
         processScheduler.setSchedulingAgent(SchedulingStrategy.EVENT_DRIVEN, eventDrivenSchedulingAgent);
 
-        final QuartzSchedulingAgent quartzSchedulingAgent = new QuartzSchedulingAgent(this, timerDrivenEngineRef.get(), contextFactory, encryptor);
-        final TimerDrivenSchedulingAgent timerDrivenAgent = new TimerDrivenSchedulingAgent(this, timerDrivenEngineRef.get(), contextFactory, encryptor, this.nifiProperties);
+        final QuartzSchedulingAgent quartzSchedulingAgent = new QuartzSchedulingAgent(this, timerDrivenEngineRef.get(), repositoryContextFactory, encryptor);
+        final TimerDrivenSchedulingAgent timerDrivenAgent = new TimerDrivenSchedulingAgent(this, timerDrivenEngineRef.get(), repositoryContextFactory, encryptor, this.nifiProperties);
         processScheduler.setSchedulingAgent(SchedulingStrategy.TIMER_DRIVEN, timerDrivenAgent);
         // PRIMARY_NODE_ONLY is deprecated, but still exists to handle processors that are still defined with it (they haven't been re-configured with executeNode = PRIMARY).
         processScheduler.setSchedulingAgent(SchedulingStrategy.PRIMARY_NODE_ONLY, timerDrivenAgent);
@@ -743,12 +744,16 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
     }
 
     public void initializeFlow() throws IOException {
+        initializeFlow(new StandardQueueProvider(getFlowManager()));
+    }
+
+    public void initializeFlow(final QueueProvider queueProvider) throws IOException {
         writeLock.lock();
         try {
             // get all connections/queues and recover from swap files.
             final List<Connection> connections = flowManager.getRootGroup().findAllConnections();
 
-            flowFileRepository.loadFlowFiles(new StandardQueueProvider(this));
+            flowFileRepository.loadFlowFiles(queueProvider);
 
             long maxIdFromSwapFiles = -1L;
             if (flowFileRepository.isVolatile()) {
@@ -1190,6 +1195,7 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
 
             for (final RemoteSiteListener listener : externalSiteListeners) {
                 listener.stop();
+                listener.destroy();
             }
 
             if (loadBalanceServer != null) {
@@ -1719,6 +1725,10 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
         return flowManager;
     }
 
+    public RepositoryContextFactory getRepositoryContextFactory() {
+        return repositoryContextFactory;
+    }
+
     /**
      * Creates a connection between two Connectable objects.
      *
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/ConnectionEventListener.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/ConnectionEventListener.java
index a3ae6ee..a5e2c5f 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/ConnectionEventListener.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/ConnectionEventListener.java
@@ -21,4 +21,14 @@ public interface ConnectionEventListener {
     void triggerSourceEvent();
 
     void triggerDestinationEvent();
+
+    ConnectionEventListener NOP_EVENT_LISTENER = new ConnectionEventListener() {
+        @Override
+        public void triggerSourceEvent() {
+        }
+
+        @Override
+        public void triggerDestinationEvent() {
+        }
+    };
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/FileSystemRepository.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/FileSystemRepository.java
index 125cd50..be7f17d 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/FileSystemRepository.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/FileSystemRepository.java
@@ -525,7 +525,7 @@ public class FileSystemRepository implements ContentRepository {
         return containerPath.resolve(resourceClaim.getSection()).resolve(resourceClaim.getId());
     }
 
-    private Path getPath(final ContentClaim claim, final boolean verifyExists) throws ContentNotFoundException {
+    public Path getPath(final ContentClaim claim, final boolean verifyExists) throws ContentNotFoundException {
         final ResourceClaim resourceClaim = claim.getResourceClaim();
         final Path containerPath = containers.get(resourceClaim.getContainer());
         if (containerPath == null) {
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/RepositoryContext.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/RepositoryContext.java
index ecf5046..44675ea 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/RepositoryContext.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/RepositoryContext.java
@@ -16,14 +16,6 @@
  */
 package org.apache.nifi.controller.repository;
 
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-import java.util.concurrent.atomic.AtomicLong;
-
 import org.apache.nifi.connectable.Connectable;
 import org.apache.nifi.connectable.ConnectableType;
 import org.apache.nifi.connectable.Connection;
@@ -32,6 +24,14 @@ import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.provenance.ProvenanceEventRepository;
 import org.apache.nifi.util.Connectables;
 
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicLong;
+
 public class RepositoryContext {
 
     private final Connectable connectable;
@@ -128,11 +128,11 @@ public class RepositoryContext {
         counterRepo.adjustCounter(globalContext, name, delta);
     }
 
-    ContentRepository getContentRepository() {
+    public ContentRepository getContentRepository() {
         return contentRepo;
     }
 
-    FlowFileRepository getFlowFileRepository() {
+    public FlowFileRepository getFlowFileRepository() {
         return flowFileRepo;
     }
 
@@ -140,7 +140,7 @@ public class RepositoryContext {
         return flowFileEventRepo;
     }
 
-    ProvenanceEventRepository getProvenanceRepository() {
+    public ProvenanceEventRepository getProvenanceRepository() {
         return provenanceRepo;
     }
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java
index 78e93a1..14cb70a 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java
@@ -614,6 +614,17 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
             }
         }
 
+        // Next, process any JOIN events because we need to ensure that the JOINed FlowFile is created before any processor-emitted events occur.
+        for (final Map.Entry<FlowFile, List<ProvenanceEventRecord>> entry : checkpoint.generatedProvenanceEvents.entrySet()) {
+            for (final ProvenanceEventRecord event : entry.getValue()) {
+                final ProvenanceEventType eventType = event.getEventType();
+                if (eventType == ProvenanceEventType.JOIN) {
+                    recordsToSubmit.add(event);
+                    addEventType(eventTypesPerFlowFileId, event.getFlowFileUuid(), event.getEventType());
+                }
+            }
+        }
+
         // Now add any Processor-reported events.
         for (final ProvenanceEventRecord event : processorGenerated) {
             if (isSpuriousForkEvent(event, checkpoint.removedFlowFiles)) {
@@ -633,6 +644,10 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
         // Finally, add any other events that we may have generated.
         for (final List<ProvenanceEventRecord> eventList : checkpoint.generatedProvenanceEvents.values()) {
             for (final ProvenanceEventRecord event : eventList) {
+                if (event.getEventType() == ProvenanceEventType.JOIN) {
+                    continue; // JOIN events are handled above.
+                }
+
                 if (isSpuriousForkEvent(event, checkpoint.removedFlowFiles)) {
                     continue;
                 }
@@ -647,17 +662,7 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
             final ContentClaim original = repoRecord.getOriginalClaim();
             final ContentClaim current = repoRecord.getCurrentClaim();
 
-            boolean contentChanged = false;
-            if (original == null && current != null) {
-                contentChanged = true;
-            }
-            if (original != null && current == null) {
-                contentChanged = true;
-            }
-            if (original != null && current != null && !original.equals(current)) {
-                contentChanged = true;
-            }
-
+            final boolean contentChanged = !Objects.equals(original, current);
             final FlowFileRecord curFlowFile = repoRecord.getCurrent();
             final String flowFileId = curFlowFile.getAttribute(CoreAttributes.UUID.key());
             boolean eventAdded = false;
@@ -1742,11 +1747,7 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
 
     private void registerJoinEvent(final FlowFile child, final Collection<FlowFile> parents) {
         final ProvenanceEventRecord eventRecord = provenanceReporter.generateJoinEvent(parents, child);
-        List<ProvenanceEventRecord> existingRecords = generatedProvenanceEvents.get(child);
-        if (existingRecords == null) {
-            existingRecords = new ArrayList<>();
-            generatedProvenanceEvents.put(child, existingRecords);
-        }
+        final List<ProvenanceEventRecord> existingRecords = generatedProvenanceEvents.computeIfAbsent(child, k -> new ArrayList<>());
         existingRecords.add(eventRecord);
     }
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardQueueProvider.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardQueueProvider.java
index 55e36c6..cd891e9 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardQueueProvider.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardQueueProvider.java
@@ -17,7 +17,7 @@
 package org.apache.nifi.controller.repository;
 
 import org.apache.nifi.connectable.Connection;
-import org.apache.nifi.controller.FlowController;
+import org.apache.nifi.controller.flow.FlowManager;
 import org.apache.nifi.controller.queue.FlowFileQueue;
 
 import java.util.ArrayList;
@@ -25,16 +25,16 @@ import java.util.Collection;
 import java.util.List;
 
 public class StandardQueueProvider implements QueueProvider {
-    private final FlowController flowController;
+    private final FlowManager flowManager;
 
-    public StandardQueueProvider(final FlowController flowController) {
-        this.flowController = flowController;
+    public StandardQueueProvider(final FlowManager flowManager) {
+        this.flowManager = flowManager;
     }
 
 
     @Override
     public Collection<FlowFileQueue> getAllQueues() {
-        final Collection<Connection> connections = flowController.getFlowManager().findAllConnections();
+        final Collection<Connection> connections = flowManager.findAllConnections();
         final List<FlowFileQueue> queues = new ArrayList<>(connections.size());
         for (final Connection connection : connections) {
             queues.add(connection.getFlowFileQueue());
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 4215633..fdde18c 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
@@ -181,6 +181,10 @@ public class WriteAheadFlowFileRepository implements FlowFileRepository, SyncLis
 
     @Override
     public void initialize(final ResourceClaimManager claimManager) throws IOException {
+        initialize(claimManager, new StandardRepositoryRecordSerdeFactory(claimManager));
+    }
+
+    protected void initialize(final ResourceClaimManager claimManager, final RepositoryRecordSerdeFactory serdeFactory) throws IOException {
         this.claimManager = claimManager;
 
         for (final File file : flowFileRepositoryPaths) {
@@ -191,7 +195,7 @@ public class WriteAheadFlowFileRepository implements FlowFileRepository, SyncLis
         // TODO: Allow for backup path that can be used if disk out of space?? Would allow a snapshot to be stored on
         // backup and then the data deleted from the normal location; then can move backup to normal location and
         // delete backup. On restore, if no files exist in partition's directory, would have to check backup directory
-        serdeFactory = new RepositoryRecordSerdeFactory(claimManager);
+        this.serdeFactory = serdeFactory;
 
         if (walImplementation.equals(SEQUENTIAL_ACCESS_WAL)) {
             wal = new SequentialAccessWriteAheadLog<>(flowFileRepositoryPaths.get(0), serdeFactory, this);
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/metrics/RingBufferEventRepository.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/metrics/RingBufferEventRepository.java
index bcd0344..629ec5e 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/metrics/RingBufferEventRepository.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/metrics/RingBufferEventRepository.java
@@ -25,7 +25,6 @@ import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 
 public class RingBufferEventRepository implements FlowFileEventRepository {
-
     private final int numMinutes;
     private final ConcurrentMap<String, EventContainer> componentEventMap = new ConcurrentHashMap<>();
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/metrics/SecondPrecisionEventContainer.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/metrics/SecondPrecisionEventContainer.java
index 2482177..b901f3f 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/metrics/SecondPrecisionEventContainer.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/metrics/SecondPrecisionEventContainer.java
@@ -28,8 +28,8 @@ public class SecondPrecisionEventContainer implements EventContainer {
 
     private final int numBins;
     private final EventSum[] sums;
-    private final EventSumValue aggregateValue = new EventSumValue(0);
-    private final AtomicLong lastUpdateSecond = new AtomicLong(0);
+    private final EventSumValue aggregateValue = new EventSumValue(0L);
+    private final AtomicLong lastUpdateSecond = new AtomicLong(System.currentTimeMillis() / 1000L);
 
     public SecondPrecisionEventContainer(final int numMinutes) {
         // number of bins is number of seconds in 'numMinutes' plus 1. We add one because
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/TimerDrivenSchedulingAgent.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/TimerDrivenSchedulingAgent.java
index db937e9..f7e4963 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/TimerDrivenSchedulingAgent.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/TimerDrivenSchedulingAgent.java
@@ -16,12 +16,6 @@
  */
 package org.apache.nifi.controller.scheduling;
 
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.ScheduledFuture;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicReference;
-
 import org.apache.nifi.connectable.Connectable;
 import org.apache.nifi.controller.FlowController;
 import org.apache.nifi.controller.ReportingTaskNode;
@@ -35,6 +29,12 @@ import org.apache.nifi.util.NiFiProperties;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+
 public class TimerDrivenSchedulingAgent extends AbstractSchedulingAgent {
 
     private static final Logger logger = LoggerFactory.getLogger(TimerDrivenSchedulingAgent.class);
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ConnectableTask.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ConnectableTask.java
index 180e2a2..45d0c8f 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ConnectableTask.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ConnectableTask.java
@@ -149,19 +149,20 @@ public class ConnectableTask {
     }
 
     public InvocationResult invoke() {
-        logger.trace("Triggering {}", connectable);
-
         if (scheduleState.isTerminated()) {
+            logger.debug("Will not trigger {} because task is terminated", connectable);
             return InvocationResult.DO_NOT_YIELD;
         }
 
         // make sure processor is not yielded
         if (isYielded()) {
+            logger.debug("Will not trigger {} because component is yielded", connectable);
             return InvocationResult.DO_NOT_YIELD;
         }
 
         // make sure that either we're not clustered or this processor runs on all nodes or that this is the primary node
         if (!isRunOnCluster(flowController)) {
+            logger.debug("Will not trigger {} because this is not the primary node", connectable);
             return InvocationResult.DO_NOT_YIELD;
         }
 
@@ -179,6 +180,8 @@ public class ConnectableTask {
             }
         }
 
+        logger.debug("Triggering {}", connectable);
+
         final long batchNanos = connectable.getRunDuration(TimeUnit.NANOSECONDS);
         final ProcessSessionFactory sessionFactory;
         final StandardProcessSession rawSession;
@@ -206,8 +209,8 @@ public class ConnectableTask {
             try (final AutoCloseable ncl = NarCloseable.withComponentNarLoader(flowController.getExtensionManager(), connectable.getRunnableComponent().getClass(), connectable.getIdentifier())) {
                 boolean shouldRun = connectable.getScheduledState() == ScheduledState.RUNNING;
                 while (shouldRun) {
-                    connectable.onTrigger(processContext, activeSessionFactory);
                     invocationCount++;
+                    connectable.onTrigger(processContext, activeSessionFactory);
 
                     if (!batch) {
                         return InvocationResult.DO_NOT_YIELD;
@@ -258,14 +261,14 @@ public class ConnectableTask {
                 if (batch) {
                     try {
                         rawSession.commit();
-                    } catch (final Exception e) {
+                    } catch (final Throwable t) {
                         final ComponentLog procLog = new SimpleProcessLogger(connectable.getIdentifier(), connectable.getRunnableComponent());
-                        procLog.error("Failed to commit session {} due to {}; rolling back", new Object[] { rawSession, e.toString() }, e);
+                        procLog.error("Failed to commit session {} due to {}; rolling back", new Object[] { rawSession, t.toString() }, t);
 
                         try {
                             rawSession.rollback(true);
                         } catch (final Exception e1) {
-                            procLog.error("Failed to roll back session {} due to {}", new Object[] { rawSession, e.toString() }, e);
+                            procLog.error("Failed to roll back session {} due to {}", new Object[] { rawSession, t.toString() }, t);
                         }
                     }
                 }
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 1bc06a1..ed123b8 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
@@ -3577,7 +3577,7 @@ public final class StandardProcessGroup implements ProcessGroup {
             if (childGroup == null) {
                 final ProcessGroup added = addProcessGroup(group, proposedChildGroup, componentIdSeed, variablesToSkip);
                 flowManager.onProcessGroupAdded(added);
-                added.findAllRemoteProcessGroups().stream().forEach(RemoteProcessGroup::initialize);
+                added.findAllRemoteProcessGroups().forEach(RemoteProcessGroup::initialize);
                 LOG.info("Added {} to {}", added, this);
             } else if (childCoordinates == null || updateDescendantVersionedGroups) {
                 updateProcessGroup(childGroup, proposedChildGroup, componentIdSeed, updatedVersionedComponentIds, true, true, updateDescendantVersionedGroups, variablesToSkip);
@@ -3695,7 +3695,7 @@ public final class StandardProcessGroup implements ProcessGroup {
 
                 final Set<Relationship> proposedAutoTerminated =
                     proposedProcessor.getAutoTerminatedRelationships() == null ? Collections.emptySet() : proposedProcessor.getAutoTerminatedRelationships().stream()
-                        .map(relName -> added.getRelationship(relName))
+                        .map(added::getRelationship)
                         .collect(Collectors.toSet());
                 autoTerminatedRelationships.put(added, proposedAutoTerminated);
                 LOG.info("Added {} to {}", added, this);
@@ -3704,7 +3704,7 @@ public final class StandardProcessGroup implements ProcessGroup {
 
                 final Set<Relationship> proposedAutoTerminated =
                     proposedProcessor.getAutoTerminatedRelationships() == null ? Collections.emptySet() : proposedProcessor.getAutoTerminatedRelationships().stream()
-                        .map(relName -> processor.getRelationship(relName))
+                        .map(processor::getRelationship)
                         .collect(Collectors.toSet());
 
                 if (!processor.getAutoTerminatedRelationships().equals(proposedAutoTerminated)) {
@@ -3778,7 +3778,7 @@ public final class StandardProcessGroup implements ProcessGroup {
         // We cannot do this above, in the 'updateProcessor' call because if a connection is removed and changed to auto-terminated,
         // then updating this in the updateProcessor call above would attempt to set the Relationship to being auto-terminated while a
         // Connection for that relationship exists. This will throw an Exception.
-        autoTerminatedRelationships.forEach((proc, rels) -> proc.setAutoTerminatedRelationships(rels));
+        autoTerminatedRelationships.forEach(ProcessorNode::setAutoTerminatedRelationships);
 
         // Remove all controller services no longer in use
         for (final String removedVersionedId : controllerServicesRemoved) {
@@ -4317,7 +4317,7 @@ public final class StandardProcessGroup implements ProcessGroup {
     }
 
     private String getServiceInstanceId(final String serviceVersionedComponentId, final ProcessGroup group) {
-        for (final ControllerServiceNode serviceNode : group.getControllerServices(true)) {
+        for (final ControllerServiceNode serviceNode : group.getControllerServices(false)) {
             final Optional<String> optionalVersionedId = serviceNode.getVersionedComponentId();
             final String versionedId = optionalVersionedId.orElseGet(() -> UUID.nameUUIDFromBytes(serviceNode.getIdentifier().getBytes(StandardCharsets.UTF_8)).toString());
             if (versionedId.equals(serviceVersionedComponentId)) {
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 a1fa975..dbb84be 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
@@ -2259,11 +2259,9 @@ public class TestStandardProcessSession {
         public Set<ContentClaim> getExistingClaims() {
             final Set<ContentClaim> claims = new HashSet<>();
 
-            for (long i = 0; i < idGenerator.get(); i++) {
-                final ResourceClaim resourceClaim = resourceClaimManager.newResourceClaim("container", "section", String.valueOf(i), false, false);
-                final ContentClaim contentClaim = new StandardContentClaim(resourceClaim, 0L);
-                if (getClaimantCount(contentClaim) > 0) {
-                    claims.add(contentClaim);
+            for (final Map.Entry<ContentClaim, AtomicInteger> entry : claimantCounts.entrySet()) {
+                if (entry.getValue().get() > 0) {
+                    claims.add(entry.getKey());
                 }
             }
 
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 cd3ee1c..4faba26 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
@@ -323,7 +323,7 @@ public class TestWriteAheadFlowFileRepository {
         assertTrue(path.toFile().mkdirs());
 
         final ResourceClaimManager claimManager = new StandardResourceClaimManager();
-        final RepositoryRecordSerdeFactory serdeFactory = new RepositoryRecordSerdeFactory(claimManager);
+        final StandardRepositoryRecordSerdeFactory serdeFactory = new StandardRepositoryRecordSerdeFactory(claimManager);
         final WriteAheadRepository<RepositoryRecord> repo = new MinimalLockingWriteAheadLog<>(path, numPartitions, serdeFactory, null);
         final Collection<RepositoryRecord> initialRecs = repo.recoverRecords();
         assertTrue(initialRecs.isEmpty());
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/AlwaysAuthorizedAuthorizer.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/AlwaysAuthorizedAuthorizer.java
new file mode 100644
index 0000000..872ac15
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/AlwaysAuthorizedAuthorizer.java
@@ -0,0 +1,56 @@
+/*
+ * 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.integration;
+
+import org.apache.nifi.authorization.AuthorizationRequest;
+import org.apache.nifi.authorization.AuthorizationResult;
+import org.apache.nifi.authorization.Authorizer;
+import org.apache.nifi.authorization.AuthorizerConfigurationContext;
+import org.apache.nifi.authorization.AuthorizerInitializationContext;
+import org.apache.nifi.authorization.exception.AuthorizationAccessException;
+import org.apache.nifi.authorization.exception.AuthorizerCreationException;
+import org.apache.nifi.authorization.exception.AuthorizerDestructionException;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+public class AlwaysAuthorizedAuthorizer implements Authorizer {
+    private List<AuthorizationRequest> requests = new ArrayList<>();
+
+    @Override
+    public AuthorizationResult authorize(final AuthorizationRequest request) throws AuthorizationAccessException {
+        requests.add(request);
+        return AuthorizationResult.approved();
+    }
+
+    @Override
+    public void initialize(final AuthorizerInitializationContext initializationContext) throws AuthorizerCreationException {
+    }
+
+    @Override
+    public void onConfigured(final AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException {
+    }
+
+    @Override
+    public void preDestruction() throws AuthorizerDestructionException {
+    }
+
+    public List<AuthorizationRequest> getAuthorizationRequests() {
+        return Collections.unmodifiableList(requests);
+    }
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/DirectInjectionExtensionManager.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/DirectInjectionExtensionManager.java
new file mode 100644
index 0000000..56cc212
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/DirectInjectionExtensionManager.java
@@ -0,0 +1,64 @@
+/*
+ * 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.integration;
+
+import org.apache.nifi.bundle.Bundle;
+import org.apache.nifi.bundle.BundleDetails;
+import org.apache.nifi.controller.ControllerService;
+import org.apache.nifi.nar.StandardExtensionDiscoveringManager;
+import org.apache.nifi.nar.SystemBundle;
+import org.apache.nifi.processor.Processor;
+import org.apache.nifi.reporting.ReportingTask;
+
+import java.io.File;
+
+public class DirectInjectionExtensionManager extends StandardExtensionDiscoveringManager {
+    private static final Bundle INTEGRATION_TEST_BUNDLE = new Bundle(new BundleDetails.Builder()
+        .workingDir(new File("target"))
+        .coordinate(SystemBundle.SYSTEM_BUNDLE_COORDINATE)
+        .build(), ClassLoader.getSystemClassLoader());
+
+    public void injectExtension(final Object extension) {
+        final Class<?> extensionType;
+        if (extension instanceof Processor) {
+            extensionType = Processor.class;
+        } else if (extension instanceof ControllerService) {
+            extensionType = ControllerService.class;
+        } else if (extension instanceof ReportingTask) {
+            extensionType = ReportingTask.class;
+        } else {
+            throw new IllegalArgumentException("Given extension is not a Processor, Controller Service, or Reporting Task");
+        }
+
+        super.loadExtension(extension, extensionType, INTEGRATION_TEST_BUNDLE);
+    }
+
+    public void injectExtensionType(final Class<?> extensionType, final String implementationClassName) {
+        final Class<?> implementationClass;
+        try {
+            implementationClass = Class.forName(implementationClassName, false, INTEGRATION_TEST_BUNDLE.getClassLoader());
+        } catch (ClassNotFoundException e) {
+            throw new IllegalArgumentException("Could not find implementation class: " + implementationClassName);
+        }
+
+        injectExtensionType(extensionType, implementationClass);
+    }
+
+    public void injectExtensionType(final Class<?> extensionType, final Class<?> implementationClass) {
+        super.registerExtensionClass(extensionType, implementationClass, INTEGRATION_TEST_BUNDLE);
+    }
+}
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
new file mode 100644
index 0000000..157f9d1
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/FrameworkIntegrationTest.java
@@ -0,0 +1,461 @@
+/*
+ * 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.integration;
+
+import org.apache.nifi.admin.service.AuditService;
+import org.apache.nifi.authorization.Authorizer;
+import org.apache.nifi.bundle.Bundle;
+import org.apache.nifi.bundle.BundleCoordinate;
+import org.apache.nifi.components.state.StateProvider;
+import org.apache.nifi.components.validation.ValidationStatus;
+import org.apache.nifi.connectable.Connection;
+import org.apache.nifi.connectable.StandardConnection;
+import org.apache.nifi.controller.ControllerService;
+import org.apache.nifi.controller.FileSystemSwapManager;
+import org.apache.nifi.controller.FlowController;
+import org.apache.nifi.controller.ProcessorNode;
+import org.apache.nifi.controller.StandardSnippet;
+import org.apache.nifi.controller.flow.StandardFlowManager;
+import org.apache.nifi.controller.queue.ConnectionEventListener;
+import org.apache.nifi.controller.queue.FlowFileQueue;
+import org.apache.nifi.controller.queue.FlowFileQueueFactory;
+import org.apache.nifi.controller.queue.LoadBalanceStrategy;
+import org.apache.nifi.controller.queue.StandardFlowFileQueue;
+import org.apache.nifi.controller.repository.ContentRepository;
+import org.apache.nifi.controller.repository.FileSystemRepository;
+import org.apache.nifi.controller.repository.FlowFileEvent;
+import org.apache.nifi.controller.repository.FlowFileEventRepository;
+import org.apache.nifi.controller.repository.FlowFileRecord;
+import org.apache.nifi.controller.repository.FlowFileRepository;
+import org.apache.nifi.controller.repository.FlowFileSwapManager;
+import org.apache.nifi.controller.repository.QueueProvider;
+import org.apache.nifi.controller.repository.RepositoryContext;
+import org.apache.nifi.controller.repository.RepositoryStatusReport;
+import org.apache.nifi.controller.repository.WriteAheadFlowFileRepository;
+import org.apache.nifi.controller.repository.claim.ResourceClaimManager;
+import org.apache.nifi.controller.repository.claim.StandardResourceClaimManager;
+import org.apache.nifi.controller.repository.metrics.RingBufferEventRepository;
+import org.apache.nifi.controller.scheduling.RepositoryContextFactory;
+import org.apache.nifi.controller.scheduling.SchedulingAgent;
+import org.apache.nifi.controller.scheduling.StandardProcessScheduler;
+import org.apache.nifi.controller.scheduling.TimerDrivenSchedulingAgent;
+import org.apache.nifi.controller.service.ControllerServiceNode;
+import org.apache.nifi.controller.service.ControllerServiceProvider;
+import org.apache.nifi.controller.service.StandardControllerServiceProvider;
+import org.apache.nifi.controller.state.providers.local.WriteAheadLocalStateProvider;
+import org.apache.nifi.controller.status.history.ComponentStatusRepository;
+import org.apache.nifi.controller.status.history.VolatileComponentStatusRepository;
+import org.apache.nifi.encrypt.StringEncryptor;
+import org.apache.nifi.engine.FlowEngine;
+import org.apache.nifi.events.VolatileBulletinRepository;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.groups.ProcessGroup;
+import org.apache.nifi.integration.processor.BiConsumerProcessor;
+import org.apache.nifi.nar.ExtensionManager;
+import org.apache.nifi.nar.SystemBundle;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Processor;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+import org.apache.nifi.provenance.ProvenanceEventRepository;
+import org.apache.nifi.provenance.ProvenanceEventType;
+import org.apache.nifi.provenance.ProvenanceRepository;
+import org.apache.nifi.provenance.WriteAheadProvenanceRepository;
+import org.apache.nifi.registry.VariableRegistry;
+import org.apache.nifi.registry.flow.FlowRegistryClient;
+import org.apache.nifi.reporting.BulletinRepository;
+import org.apache.nifi.scheduling.SchedulingStrategy;
+import org.apache.nifi.util.FileUtils;
+import org.apache.nifi.util.NiFiProperties;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.rules.Timeout;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.BiConsumer;
+
+import static org.junit.Assert.assertEquals;
+
+public class FrameworkIntegrationTest {
+    //@Rule
+    public Timeout globalTimeout = Timeout.seconds(20);
+
+    private ResourceClaimManager resourceClaimManager;
+    private StandardProcessScheduler processScheduler;
+
+    private FlowEngine flowEngine;
+    private FlowController flowController;
+    private FlowRegistryClient flowRegistryClient = null;
+    private ProcessorNode nopProcessor;
+    private ProcessorNode terminateProcessor;
+    private ProcessorNode terminateAllProcessor;
+    private FlowFileQueueFactory flowFileQueueFactory;
+    private FlowFileSwapManager flowFileSwapManager;
+    private DirectInjectionExtensionManager extensionManager;
+    private ProcessGroup rootProcessGroup;
+    private Bundle systemBundle;
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder().name("success").build();
+
+    @Before
+    public void setup() throws IOException {
+        cleanup();
+        initialize(null);
+    }
+
+    protected String getNiFiPropertiesFilename() {
+        return "src/test/resources/int-tests/default-nifi.properties";
+    }
+
+    protected Map<String, String> getNiFiPropertiesOverrides() {
+        return Collections.emptyMap();
+    }
+
+    protected void injectExtensionTypes(final DirectInjectionExtensionManager extensionManager) {
+        // Placeholder for subclasses.
+    }
+
+    protected final void initialize(final QueueProvider queueProvider) throws IOException {
+        final NiFiProperties nifiProperties = NiFiProperties.createBasicNiFiProperties(getNiFiPropertiesFilename(), getNiFiPropertiesOverrides());
+        initialize(nifiProperties, queueProvider);
+    }
+
+    protected final void initialize(final NiFiProperties nifiProperties, final QueueProvider queueProvider) throws IOException {
+        final FlowFileEventRepository flowFileEventRepository = new RingBufferEventRepository(5);
+        resourceClaimManager = new StandardResourceClaimManager();
+
+        final BulletinRepository bulletinRepo = new VolatileBulletinRepository();
+        flowEngine = new FlowEngine(4, "unit test flow engine");
+        extensionManager = new DirectInjectionExtensionManager();
+
+        extensionManager.injectExtensionType(FlowFileRepository.class, WriteAheadFlowFileRepository.class);
+        extensionManager.injectExtensionType(ContentRepository.class, FileSystemRepository.class);
+        extensionManager.injectExtensionType(ProvenanceRepository.class, WriteAheadProvenanceRepository.class);
+        extensionManager.injectExtensionType(StateProvider.class, WriteAheadLocalStateProvider.class);
+        extensionManager.injectExtensionType(ComponentStatusRepository.class, VolatileComponentStatusRepository.class);
+        extensionManager.injectExtensionType(FlowFileSwapManager.class, FileSystemSwapManager.class);
+        extensionManager.injectExtensionType(Processor.class, BiConsumerProcessor.class);
+
+        injectExtensionTypes(extensionManager);
+        systemBundle = SystemBundle.create(nifiProperties);
+        extensionManager.discoverExtensions(systemBundle, Collections.emptySet());
+
+        final StringEncryptor encryptor = StringEncryptor.createEncryptor("PBEWITHMD5AND256BITAES-CBC-OPENSSL", "BC", "unit-test");
+        final Authorizer authorizer = new AlwaysAuthorizedAuthorizer();
+        final AuditService auditService = new NopAuditService();
+
+        flowController = FlowController.createStandaloneInstance(flowFileEventRepository, nifiProperties, authorizer, auditService, encryptor, bulletinRepo,
+            VariableRegistry.ENVIRONMENT_SYSTEM_REGISTRY, flowRegistryClient, extensionManager);
+
+        processScheduler = new StandardProcessScheduler(flowEngine, flowController, encryptor, flowController.getStateManagerProvider(), nifiProperties);
+
+        final RepositoryContextFactory repositoryContextFactory = flowController.getRepositoryContextFactory();
+        final SchedulingAgent timerDrivenSchedulingAgent = new TimerDrivenSchedulingAgent(flowController, flowEngine, repositoryContextFactory, encryptor, nifiProperties);
+        processScheduler.setSchedulingAgent(SchedulingStrategy.TIMER_DRIVEN, timerDrivenSchedulingAgent);
+
+        final ControllerServiceProvider controllerServiceProvider = new StandardControllerServiceProvider(flowController, processScheduler, bulletinRepo);
+
+        rootProcessGroup = flowController.getFlowManager().createProcessGroup(UUID.randomUUID().toString());
+        ((StandardFlowManager) flowController.getFlowManager()).setRootGroup(rootProcessGroup);
+
+        nopProcessor = createProcessorNode((context, session) -> {});
+
+        terminateProcessor = createProcessorNode((context, session) -> {
+            FlowFile flowFile = session.get();
+            if (flowFile == null) {
+                return;
+            }
+
+            session.remove(flowFile);
+        });
+
+        terminateAllProcessor = createProcessorNode((context, session) -> {
+            FlowFile flowFile;
+            while ((flowFile = session.get()) != null) {
+                session.remove(flowFile);
+            }
+        });
+
+        flowFileSwapManager = flowController.createSwapManager();
+        flowFileQueueFactory = new FlowFileQueueFactory() {
+            @Override
+            public FlowFileQueue createFlowFileQueue(final LoadBalanceStrategy loadBalanceStrategy, final String partitioningAttribute, final ConnectionEventListener connectionEventListener) {
+                return FrameworkIntegrationTest.this.createFlowFileQueue(UUID.randomUUID().toString());
+            }
+        };
+
+        if (queueProvider == null) {
+            flowController.initializeFlow();
+        } else {
+            flowController.initializeFlow(queueProvider);
+        }
+    }
+
+    @After
+    public final void shutdown() {
+        flowController.shutdown(true);
+        flowEngine.shutdownNow();
+        processScheduler.shutdown();
+    }
+
+    @After
+    public final void cleanup() throws IOException {
+        deleteDirectory(new File("target/int-tests"));
+    }
+
+    private void deleteDirectory(final File dir) throws IOException {
+        if (!dir.exists()) {
+            return;
+        }
+
+        FileUtils.deleteFile(dir, true);
+    }
+
+    protected FlowFileQueue createFlowFileQueue(final String uuid) {
+        final RepositoryContext repoContext = getRepositoryContext();
+        return new StandardFlowFileQueue(uuid, ConnectionEventListener.NOP_EVENT_LISTENER, repoContext.getFlowFileRepository(), repoContext.getProvenanceRepository(),
+            resourceClaimManager, processScheduler, flowFileSwapManager, flowController.createEventReporter(), 20000, 10000L, "1 GB");
+    }
+
+    protected final ProcessorNode createProcessorNode(final Class<? extends Processor> processorType) {
+        return createProcessorNode(processorType.getName());
+    }
+
+    protected final ProcessorNode createProcessorNode(final String processorType) {
+        final String uuid = getSimpleTypeName(processorType) + "-" + UUID.randomUUID().toString();
+        final BundleCoordinate bundleCoordinate = SystemBundle.SYSTEM_BUNDLE_COORDINATE;
+        final ProcessorNode procNode = flowController.getFlowManager().createProcessor(processorType, uuid, bundleCoordinate, Collections.emptySet(), true, true);
+        rootProcessGroup.addProcessor(procNode);
+
+        return procNode;
+    }
+
+    protected final ControllerServiceNode createControllerServiceNode(final Class<? extends ControllerService> controllerServiceType) {
+        return createControllerServiceNode(controllerServiceType.getName());
+    }
+
+    protected final ControllerServiceNode createControllerServiceNode(final String controllerServiceType) {
+        final String uuid = getSimpleTypeName(controllerServiceType) + "-" + UUID.randomUUID().toString();
+        final BundleCoordinate bundleCoordinate = SystemBundle.SYSTEM_BUNDLE_COORDINATE;
+        final ControllerServiceNode serviceNode = flowController.getFlowManager().createControllerService(controllerServiceType, uuid, bundleCoordinate, Collections.emptySet(), true, true);
+        rootProcessGroup.addControllerService(serviceNode);
+        return serviceNode;
+    }
+
+    private String getSimpleTypeName(final String className) {
+        final int index = className.lastIndexOf(".");
+        if (index >= 0 && index < className.length()) {
+            return className.substring(index + 1);
+        } else {
+            return "";
+        }
+    }
+
+    protected ProcessGroup getRootGroup() {
+        return rootProcessGroup;
+    }
+
+    /**
+     * Creates a Processor that is responsible for generating a FlowFile of the given size and routing to "success".
+     *
+     * @param contentSize the number of bytes for the content
+     *
+     * @return the ProcessorNode
+     */
+    protected final ProcessorNode createGenerateProcessor(final int contentSize) {
+        return createGenerateProcessor(contentSize, null);
+    }
+
+    /**
+     * Creates a Processor that is responsible for generating a FlowFile of the given size and routing to "success". The generated FlowFile is set in the given AtomicReference
+     *
+     * @param contentSize the number of bytes for the content
+     * @param flowFileReference an AtomicReference to hold the flowfile
+     *
+     * @return the ProcessorNode
+     */
+    protected final ProcessorNode createGenerateProcessor(final int contentSize, final AtomicReference<FlowFileRecord> flowFileReference) {
+        return createProcessorNode((context, session) -> {
+            FlowFile flowFile = session.create();
+            flowFile = session.write(flowFile, out -> out.write(new byte[contentSize]));
+
+            if (flowFileReference != null) {
+                flowFileReference.set((FlowFileRecord) flowFile);
+            }
+
+            session.transfer(flowFile, REL_SUCCESS);
+        }, REL_SUCCESS);
+    }
+
+    protected final ProcessorNode createProcessorNode(final BiConsumer<ProcessContext, ProcessSession> trigger, final Relationship... relationships) {
+        final Set<Relationship> relationshipSet = new HashSet<>(Arrays.asList(relationships));
+
+        final ProcessorNode processorNode = createProcessorNode(BiConsumerProcessor.class.getName());
+        final BiConsumerProcessor biConsumerProcessor = (BiConsumerProcessor) processorNode.getProcessor();
+        biConsumerProcessor.setRelationships(relationshipSet);
+        biConsumerProcessor.setTrigger(trigger);
+
+        return processorNode;
+    }
+
+    protected final void connect(final ProcessorNode source, final ProcessorNode destination, final Relationship relationship) {
+        connect(source, destination, Collections.singleton(relationship));
+    }
+
+    protected final void connect(final ProcessorNode source, final ProcessorNode destination, final Collection<Relationship> relationships) {
+        final Connection connection = new StandardConnection.Builder(processScheduler)
+            .source(source)
+            .destination(destination)
+            .relationships(relationships)
+            .id(UUID.randomUUID().toString())
+            .clustered(false)
+            .flowFileQueueFactory(flowFileQueueFactory)
+            .build();
+
+        source.addConnection(connection);
+        destination.addConnection(connection);
+        rootProcessGroup.addConnection(connection);
+    }
+
+    protected final Future<Void> start(final ProcessorNode procNode) {
+        final ValidationStatus validationStatus = procNode.performValidation();
+        if (validationStatus != ValidationStatus.VALID) {
+            throw new IllegalStateException("Processor is invalid: " + procNode + ": " + procNode.getValidationErrors());
+        }
+
+        return rootProcessGroup.startProcessor(procNode, true);
+    }
+
+    protected final Future<Void> stop(final ProcessorNode procNode) {
+        return rootProcessGroup.stopProcessor(procNode);
+    }
+
+    protected final FlowFileQueue getDestinationQueue(final ProcessorNode procNode, final Relationship relationship) {
+        return procNode.getConnections(relationship).stream()
+            .map(Connection::getFlowFileQueue)
+            .findAny()
+            .orElseThrow(() -> new IllegalArgumentException("Could not find queue for relationship with name <" + relationship + ">"));
+    }
+
+    protected final FlowFileRepository getFlowFileRepository() {
+        return getRepositoryContext().getFlowFileRepository();
+    }
+
+    protected Bundle getSystemBundle() {
+        return systemBundle;
+    }
+
+    protected final ContentRepository getContentRepository() {
+        return getRepositoryContext().getContentRepository();
+    }
+
+    protected final ProvenanceEventRepository getProvenanceRepository() {
+        return getRepositoryContext().getProvenanceRepository();
+    }
+
+    private RepositoryContext getRepositoryContext() {
+        return flowController.getRepositoryContextFactory().newProcessContext(nopProcessor, new AtomicLong(0L));
+    }
+
+    protected final ProcessorNode getNopProcessor() {
+        return nopProcessor;
+    }
+
+    protected final ProcessorNode getTerminateProcessor() {
+        return terminateProcessor;
+    }
+
+    protected final ProcessorNode getTerminateAllProcessor() {
+        return terminateAllProcessor;
+    }
+
+    protected final FlowController getFlowController() {
+        return flowController;
+    }
+
+    protected void assertProvenanceEventCount(final ProvenanceEventType eventType, final int count) throws IOException {
+        int encountered = 0;
+
+        for (final ProvenanceEventRecord event : getProvenanceRepository().getEvents(0L, 100_000_000)) {
+            if (event.getEventType() == eventType) {
+                encountered++;
+            }
+        }
+
+        assertEquals("Expected to encounter " + count + " Provenance Events of type " + eventType + " but encountered " + encountered, count, encountered);
+    }
+
+    protected void triggerOnce(final ProcessorNode processor) throws ExecutionException, InterruptedException {
+        final String schedulingPeriod = processor.getSchedulingPeriod();
+        try {
+            final FlowFileEvent initialReport = getStatusReport(processor);
+            final int initialInvocations = (initialReport == null) ? 0 : initialReport.getInvocations();
+
+            processor.setScheduldingPeriod("1 hour");
+
+            // We will only trigger the Processor to run once per hour. So we need to ensure that
+            // we don't trigger the Processor while it's yielded. So if its yield expiration is in the future,
+            // wait until the yield expires.
+            while (processor.getYieldExpiration() > System.currentTimeMillis()) {
+                Thread.sleep(1L);
+            }
+
+            start(processor).get();
+
+            int totalInvocations = initialInvocations;
+            while (totalInvocations < initialInvocations + 1) {
+                final FlowFileEvent currentReport = getStatusReport(processor);
+                totalInvocations = currentReport == null ? 0 : currentReport.getInvocations();
+            }
+
+            stop(processor).get();
+        } finally {
+            processor.setScheduldingPeriod(schedulingPeriod);
+        }
+    }
+
+    protected FlowFileEvent getStatusReport(final ProcessorNode processor) {
+        final FlowFileEventRepository repository = getRepositoryContext().getFlowFileEventRepository();
+        RepositoryStatusReport statusReport = repository.reportTransferEvents(0L);
+        return statusReport.getReportEntry(processor.getIdentifier());
+    }
+
+    protected void moveProcessor(final ProcessorNode processor, final ProcessGroup destination) {
+        final StandardSnippet snippet = new StandardSnippet();
+        snippet.setParentGroupId(processor.getProcessGroupIdentifier());
+        snippet.addProcessors(Collections.singletonMap(processor.getIdentifier(), null));
+
+        processor.getProcessGroup().move(snippet, destination);
+    }
+
+    protected ExtensionManager getExtensionManager() {
+        return extensionManager;
+    }
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/NopAuditService.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/NopAuditService.java
new file mode 100644
index 0000000..878dc38
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/NopAuditService.java
@@ -0,0 +1,60 @@
+/*
+ * 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.integration;
+
+import org.apache.nifi.action.Action;
+import org.apache.nifi.admin.service.AuditService;
+import org.apache.nifi.history.History;
+import org.apache.nifi.history.HistoryQuery;
+import org.apache.nifi.history.PreviousValue;
+
+import java.util.Collection;
+import java.util.Date;
+import java.util.List;
+import java.util.Map;
+
+public class NopAuditService implements AuditService {
+    @Override
+    public void addActions(final Collection<Action> actions) {
+
+    }
+
+    @Override
+    public Map<String, List<PreviousValue>> getPreviousValues(final String componentId) {
+        return null;
+    }
+
+    @Override
+    public History getActions(final HistoryQuery actionQuery) {
+        return null;
+    }
+
+    @Override
+    public History getActions(final int firstActionId, final int maxActions) {
+        return null;
+    }
+
+    @Override
+    public Action getAction(final Integer actionId) {
+        return null;
+    }
+
+    @Override
+    public void purgeActions(final Date end, final Action purgeAction) {
+
+    }
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/cs/ControllerServiceReferenceIT.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/cs/ControllerServiceReferenceIT.java
new file mode 100644
index 0000000..b586444
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/cs/ControllerServiceReferenceIT.java
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.integration.cs;
+
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.components.validation.ValidationStatus;
+import org.apache.nifi.controller.ControllerService;
+import org.apache.nifi.controller.ProcessorNode;
+import org.apache.nifi.controller.service.ControllerServiceNode;
+import org.apache.nifi.integration.DirectInjectionExtensionManager;
+import org.apache.nifi.integration.FrameworkIntegrationTest;
+import org.apache.nifi.processor.Processor;
+import org.junit.Test;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.concurrent.ExecutionException;
+
+import static org.junit.Assert.assertEquals;
+import static org.testng.Assert.assertSame;
+
+public class ControllerServiceReferenceIT extends FrameworkIntegrationTest {
+    @Override
+    protected void injectExtensionTypes(final DirectInjectionExtensionManager extensionManager) {
+        extensionManager.injectExtensionType(ControllerService.class, CounterControllerService.class);
+        extensionManager.injectExtensionType(ControllerService.class, LongValidatingControllerService.class);
+        extensionManager.injectExtensionType(Processor.class, ControllerServiceReferencingProcessor.class);
+    }
+
+
+    @Test
+    public void testCallingControllerService() throws ExecutionException, InterruptedException {
+        final ProcessorNode counter = createProcessorNode(ControllerServiceReferencingProcessor.class.getName());
+
+        final ControllerServiceNode serviceNode = createControllerServiceNode(CounterControllerService.class.getName());
+        assertSame(ValidationStatus.VALID, serviceNode.performValidation());
+        getFlowController().getControllerServiceProvider().enableControllerService(serviceNode).get();
+
+        counter.setAutoTerminatedRelationships(Collections.singleton(REL_SUCCESS));
+        counter.setProperties(Collections.singletonMap("Counter Service", serviceNode.getIdentifier()));
+
+        triggerOnce(counter);
+
+        assertEquals(1, ((Counter) serviceNode.getControllerServiceImplementation()).getValue());
+    }
+
+    @Test
+    public void testLongValidatingControllerService() {
+        final ControllerServiceNode serviceNode = createControllerServiceNode(LongValidatingControllerService.class.getName());
+        serviceNode.setProperties(Collections.singletonMap(LongValidatingControllerService.DELAY.getName(), "250 millis"));
+        final ValidationStatus validationStatus = serviceNode.performValidation();
+        final Collection<ValidationResult> validationErrors = serviceNode.getValidationErrors();
+        assertSame(validationStatus, ValidationStatus.VALID);
+        assertEquals(0, validationErrors.size());
+    }
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/cs/ControllerServiceReferencingProcessor.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/cs/ControllerServiceReferencingProcessor.java
new file mode 100644
index 0000000..82438e8
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/cs/ControllerServiceReferencingProcessor.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.integration.cs;
+
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.integration.FrameworkIntegrationTest;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+
+public class ControllerServiceReferencingProcessor extends AbstractProcessor {
+    private final PropertyDescriptor SERVICE = new PropertyDescriptor.Builder()
+        .name("Counter Service")
+        .identifiesControllerService(Counter.class)
+        .required(true)
+        .build();
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return Collections.singleton(FrameworkIntegrationTest.REL_SUCCESS);
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return Collections.singletonList(SERVICE);
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        context.getProperty(SERVICE).asControllerService(Counter.class).increment(1L);
+    }
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/ConnectionEventListener.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/cs/Counter.java
similarity index 80%
copy from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/ConnectionEventListener.java
copy to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/cs/Counter.java
index a3ae6ee..6a46400 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/ConnectionEventListener.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/cs/Counter.java
@@ -14,11 +14,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+package org.apache.nifi.integration.cs;
 
-package org.apache.nifi.controller.queue;
+import org.apache.nifi.controller.ControllerService;
 
-public interface ConnectionEventListener {
-    void triggerSourceEvent();
+public interface Counter extends ControllerService {
+    long increment(long delta);
 
-    void triggerDestinationEvent();
+    long getValue();
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/RemoteSiteListener.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/cs/CounterControllerService.java
similarity index 62%
copy from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/RemoteSiteListener.java
copy to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/cs/CounterControllerService.java
index 1183fc5..5d49466 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/RemoteSiteListener.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/cs/CounterControllerService.java
@@ -14,17 +14,22 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.nifi.remote;
+package org.apache.nifi.integration.cs;
 
-import java.io.IOException;
+import org.apache.nifi.controller.AbstractControllerService;
 
-import org.apache.nifi.groups.ProcessGroup;
+import java.util.concurrent.atomic.AtomicLong;
 
-public interface RemoteSiteListener {
+public class CounterControllerService extends AbstractControllerService implements Counter {
+    private final AtomicLong count = new AtomicLong(0L);
 
-    void setRootGroup(ProcessGroup rootGroup);
+    @Override
+    public long increment(final long delta) {
+        return count.addAndGet(delta);
+    }
 
-    void start() throws IOException;
-
-    void stop();
+    @Override
+    public long getValue() {
+        return count.get();
+    }
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/cs/LongValidatingControllerService.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/cs/LongValidatingControllerService.java
new file mode 100644
index 0000000..4bad93a
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/cs/LongValidatingControllerService.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.integration.cs;
+
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.PropertyDescriptor.Builder;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.processor.util.StandardValidators;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.nifi.expression.ExpressionLanguageScope.NONE;
+
+public class LongValidatingControllerService extends AbstractControllerService {
+    public static final PropertyDescriptor DELAY = new Builder()
+        .name("Delay")
+        .displayName("Delay")
+        .description("Amount of time to sleep during validation")
+        .required(true)
+        .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+        .expressionLanguageSupported(NONE)
+        .defaultValue("2 sec")
+        .build();
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return Collections.singletonList(DELAY);
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
+        final long millis = validationContext.getProperty(DELAY).asTimePeriod(TimeUnit.MILLISECONDS);
+        try {
+            Thread.sleep(millis);
+        } catch (final InterruptedException ie) {
+        }
+
+        return Collections.emptyList();
+    }
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/ConnectionEventListener.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/cs/NopControllerService.java
similarity index 83%
copy from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/ConnectionEventListener.java
copy to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/cs/NopControllerService.java
index a3ae6ee..2dcc5aa 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/ConnectionEventListener.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/cs/NopControllerService.java
@@ -14,11 +14,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+package org.apache.nifi.integration.cs;
 
-package org.apache.nifi.controller.queue;
+import org.apache.nifi.controller.ControllerService;
 
-public interface ConnectionEventListener {
-    void triggerSourceEvent();
-
-    void triggerDestinationEvent();
+public interface NopControllerService extends ControllerService {
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/cs/NopServiceReferencingProcessor.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/cs/NopServiceReferencingProcessor.java
new file mode 100644
index 0000000..8c55326
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/cs/NopServiceReferencingProcessor.java
@@ -0,0 +1,46 @@
+/*
+ * 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.integration.cs;
+
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.PropertyDescriptor.Builder;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+
+import java.util.Collections;
+import java.util.List;
+
+public class NopServiceReferencingProcessor extends AbstractProcessor {
+    public static final PropertyDescriptor SERVICE = new Builder()
+        .name("Controller Service")
+        .displayName("Controller Service")
+        .description("The Controller Service")
+        .required(true)
+        .identifiesControllerService(NopControllerService.class)
+        .build();
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return Collections.singletonList(SERVICE);
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+    }
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/flowfilerepo/OOMEFlowFileRepoUpdateIT.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/flowfilerepo/OOMEFlowFileRepoUpdateIT.java
new file mode 100644
index 0000000..64a7657
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/flowfilerepo/OOMEFlowFileRepoUpdateIT.java
@@ -0,0 +1,76 @@
+/*
+ * 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.integration.flowfilerepo;
+
+import org.apache.nifi.controller.ProcessorNode;
+import org.apache.nifi.controller.repository.FlowFileRepository;
+import org.apache.nifi.controller.repository.WriteAheadFlowFileRepository;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.integration.DirectInjectionExtensionManager;
+import org.apache.nifi.integration.FrameworkIntegrationTest;
+import org.apache.nifi.reporting.Bulletin;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ExecutionException;
+
+import static org.junit.Assert.assertEquals;
+
+public class OOMEFlowFileRepoUpdateIT extends FrameworkIntegrationTest {
+    @Override
+    protected Map<String, String> getNiFiPropertiesOverrides() {
+        return Collections.singletonMap("nifi.flowfile.repository.implementation", OOMEWriteAheadFlowFileRepository.class.getName());
+    }
+
+    @Override
+    protected void injectExtensionTypes(final DirectInjectionExtensionManager extensionManager) {
+        extensionManager.injectExtensionType(FlowFileRepository.class, OOMEWriteAheadFlowFileRepository.class);
+    }
+
+    @Test
+    public void testOOMEOnUpdatePreventsSubsequentUpdates() throws ExecutionException, InterruptedException, IOException {
+        final ProcessorNode generate = createProcessorNode((context, session) -> {
+            FlowFile flowFile = session.create();
+            session.transfer(flowFile, REL_SUCCESS);
+        }, REL_SUCCESS);
+        connect(generate, getTerminateAllProcessor(), REL_SUCCESS);
+
+        for (int i=0; i < 4; i++) {
+            triggerOnce(generate);
+        }
+
+        List<Bulletin> processorBulletins = getFlowController().getBulletinRepository().findBulletinsForSource(generate.getIdentifier());
+        assertEquals(1, processorBulletins.size());
+
+        triggerOnce(generate);
+
+        // FlowFile Repository should not allow us to udpate until it has been checkpointed.
+        processorBulletins = getFlowController().getBulletinRepository().findBulletinsForSource(generate.getIdentifier());
+        assertEquals(2, processorBulletins.size());
+
+        // Checkpoint the repository.
+        ((WriteAheadFlowFileRepository) getFlowFileRepository()).checkpoint();
+
+        // Should now succeed.
+        triggerOnce(generate);
+        processorBulletins = getFlowController().getBulletinRepository().findBulletinsForSource(generate.getIdentifier());
+        assertEquals(2, processorBulletins.size());
+    }
+}
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
new file mode 100644
index 0000000..1509e41
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/flowfilerepo/OOMEWriteAheadFlowFileRepository.java
@@ -0,0 +1,142 @@
+/*
+ * 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.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.StandardRepositoryRecordSerdeFactory;
+import org.apache.nifi.controller.repository.WriteAheadFlowFileRepository;
+import org.apache.nifi.controller.repository.claim.ResourceClaimManager;
+import org.apache.nifi.util.NiFiProperties;
+import org.wali.SerDe;
+import org.wali.UpdateType;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.Map;
+
+public class OOMEWriteAheadFlowFileRepository extends WriteAheadFlowFileRepository {
+
+    public OOMEWriteAheadFlowFileRepository() {
+    }
+
+    public OOMEWriteAheadFlowFileRepository(final NiFiProperties nifiProperties) {
+        super(nifiProperties);
+    }
+
+    @Override
+    public void initialize(final ResourceClaimManager claimManager) throws IOException {
+        super.initialize(claimManager, new ThrowOOMERepositoryRecordSerdeFactory(new StandardRepositoryRecordSerdeFactory(claimManager)));
+    }
+
+
+    private static class ThrowOOMERepositoryRecordSerdeFactory implements RepositoryRecordSerdeFactory {
+        private final RepositoryRecordSerdeFactory factory;
+
+        public ThrowOOMERepositoryRecordSerdeFactory(final RepositoryRecordSerdeFactory factory) {
+            this.factory = factory;
+        }
+
+        @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);
+            return new ThrowOOMESerde(serde, 3);
+        }
+
+        @Override
+        public Long getRecordIdentifier(final RepositoryRecord record) {
+            return factory.getRecordIdentifier(record);
+        }
+
+        @Override
+        public UpdateType getUpdateType(final RepositoryRecord record) {
+            return factory.getUpdateType(record);
+        }
+
+        @Override
+        public String getLocation(final RepositoryRecord record) {
+            return factory.getLocation(record);
+        }
+    }
+
+
+    private static class ThrowOOMESerde implements SerDe<RepositoryRecord> {
+        private final SerDe<RepositoryRecord> serde;
+        private final int afterSuccessfulAttempts;
+        private int successfulUpdates = 0;
+
+        public ThrowOOMESerde(final SerDe<RepositoryRecord> 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 {
+            if (successfulUpdates++ == afterSuccessfulAttempts) {
+                throw new OutOfMemoryError("Intentional OOME for unit test");
+            }
+
+            serde.serializeEdit(previousRecordState, newRecordState, out);
+        }
+
+        @Override
+        public void serializeRecord(final RepositoryRecord record, final DataOutputStream out) throws IOException {
+            if (successfulUpdates++ == afterSuccessfulAttempts) {
+                throw new OutOfMemoryError("Intentional OOME for unit test");
+            }
+
+            serde.serializeRecord(record, out);
+        }
+
+        @Override
+        public RepositoryRecord deserializeEdit(final DataInputStream in, final Map<Object, RepositoryRecord> currentRecordStates, final int version) throws IOException {
+            return serde.deserializeEdit(in, currentRecordStates, version);
+        }
+
+        @Override
+        public RepositoryRecord deserializeRecord(final DataInputStream in, final int version) throws IOException {
+            return serde.deserializeRecord(in, version);
+        }
+
+        @Override
+        public Object getRecordIdentifier(final RepositoryRecord record) {
+            return serde.getRecordIdentifier(record);
+        }
+
+        @Override
+        public UpdateType getUpdateType(final RepositoryRecord record) {
+            return serde.getUpdateType(record);
+        }
+
+        @Override
+        public String getLocation(final RepositoryRecord record) {
+            return serde.getLocation(record);
+        }
+
+        @Override
+        public int getVersion() {
+            return serde.getVersion();
+        }
+    }
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/lifecycle/ContentCleanupIT.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/lifecycle/ContentCleanupIT.java
new file mode 100644
index 0000000..90f1f3d
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/lifecycle/ContentCleanupIT.java
@@ -0,0 +1,184 @@
+/*
+ * 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.integration.lifecycle;
+
+import org.apache.nifi.controller.ProcessorNode;
+import org.apache.nifi.controller.repository.FileSystemRepository;
+import org.apache.nifi.controller.repository.FlowFileRecord;
+import org.apache.nifi.controller.repository.WriteAheadFlowFileRepository;
+import org.apache.nifi.controller.repository.claim.ContentClaim;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.integration.FrameworkIntegrationTest;
+import org.apache.nifi.provenance.ProvenanceEventType;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static junit.framework.TestCase.assertTrue;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotEquals;
+
+public class ContentCleanupIT extends FrameworkIntegrationTest {
+
+    @Test
+    public void testCompletedContentClaimCleanedUpOnCheckpoint() throws InterruptedException, IOException, ExecutionException {
+        final AtomicReference<FlowFileRecord> largeFlowFileReference = new AtomicReference<>();
+        final AtomicReference<FlowFileRecord> smallFlowFileReference = new AtomicReference<>();
+
+        // Processor to write 1 MB of content to a FlowFile
+        final ProcessorNode createLargeProcessor = createGenerateProcessor(1024 * 1024, largeFlowFileReference);
+        final ProcessorNode createSmallProcessor = createGenerateProcessor(5, smallFlowFileReference);
+
+        connect(createLargeProcessor, getTerminateProcessor(), REL_SUCCESS);
+        connect(createSmallProcessor, getTerminateProcessor(), REL_SUCCESS);
+
+        // Trigger the create processor.
+        triggerOnce(createLargeProcessor);
+        triggerOnce(createSmallProcessor);
+
+        // Ensure content available and has a claim count of 1.
+        final ContentClaim largeContentClaim = largeFlowFileReference.get().getContentClaim();
+        final ContentClaim smallContentClaim = smallFlowFileReference.get().getContentClaim();
+        assertNotEquals(largeContentClaim.getResourceClaim(), smallContentClaim.getResourceClaim());
+        assertEquals(1, getContentRepository().getClaimantCount(largeContentClaim));
+        assertEquals(1, getContentRepository().getClaimantCount(smallContentClaim));
+
+        // Ensure that content is still available and considered 'in use'
+        final FileSystemRepository fileSystemRepository = (FileSystemRepository) getContentRepository();
+        final Path largeClaimPath = fileSystemRepository.getPath(largeContentClaim, false);
+        final Path smallClaimPath = fileSystemRepository.getPath(smallContentClaim, false);
+        assertTrue(Files.exists(largeClaimPath));
+        assertTrue(largeContentClaim.getResourceClaim().isInUse());
+        assertTrue(Files.exists(smallClaimPath));
+        assertTrue(smallContentClaim.getResourceClaim().isInUse());
+
+        int recordCount = ((WriteAheadFlowFileRepository) getFlowFileRepository()).checkpoint();
+        assertEquals(2, recordCount);
+
+        // Trigger the delete Processor.
+        triggerOnce(getTerminateProcessor());
+        triggerOnce(getTerminateProcessor());
+
+        // Claim count should now be 0 and resource claim should not be in use.
+        assertEquals(0, getContentRepository().getClaimantCount(largeContentClaim));
+        assertEquals(0, getContentRepository().getClaimantCount(largeContentClaim));
+
+        assertFalse(largeContentClaim.getResourceClaim().isInUse());
+        assertTrue(smallContentClaim.getResourceClaim().isInUse());
+
+        // Checkpoint the FlowFile Repo
+        recordCount = ((WriteAheadFlowFileRepository) getFlowFileRepository()).checkpoint();
+        assertEquals(0, recordCount);
+
+        // Wait for the data to be deleted/archived.
+        waitForClaimDestruction(largeContentClaim);
+
+        assertTrue(Files.exists(smallClaimPath));
+
+        assertProvenanceEventCount(ProvenanceEventType.CREATE, 2);
+        assertProvenanceEventCount(ProvenanceEventType.DROP, 2);
+    }
+
+
+    @Test
+    public void testTransientClaimsNotHeld() throws ExecutionException, InterruptedException, IOException {
+        final AtomicReference<ContentClaim> claimReference = new AtomicReference<>();
+
+        final ProcessorNode processor = createProcessorNode((context, session) -> {
+            FlowFile flowFile = session.create();
+
+            for (int i=0; i < 1000; i++) {
+                final byte[] bytes = String.valueOf(i).getBytes();
+                flowFile = session.write(flowFile, out -> out.write(bytes));
+            }
+
+            // Write 1 MB to the content claim in order to ensure that the claim is no longer usable.
+            final byte[] oneMB = new byte[1024 * 1024];
+            flowFile = session.write(flowFile, out -> out.write(oneMB));
+
+            claimReference.set(((FlowFileRecord) flowFile).getContentClaim());
+            session.transfer(flowFile, REL_SUCCESS);
+
+        }, REL_SUCCESS);
+
+        connect(processor, getTerminateProcessor(), REL_SUCCESS);
+        triggerOnce(processor);
+
+        final int claimCount = getContentRepository().getClaimantCount(claimReference.get());
+        assertEquals(1, claimCount);
+
+        int recordCount = ((WriteAheadFlowFileRepository) getFlowFileRepository()).checkpoint();
+        assertEquals(1, recordCount);
+        assertTrue(claimReference.get().getResourceClaim().isInUse());
+
+        triggerOnce(getTerminateProcessor());
+        assertFalse(claimReference.get().getResourceClaim().isInUse());
+
+        recordCount = ((WriteAheadFlowFileRepository) getFlowFileRepository()).checkpoint();
+        assertEquals(0, recordCount);
+
+        waitForClaimDestruction(claimReference.get());
+
+        assertProvenanceEventCount(ProvenanceEventType.CREATE, 1);
+        assertProvenanceEventCount(ProvenanceEventType.DROP, 1);
+    }
+
+
+    @Test
+    public void testCloneIncrementsContentClaim() throws ExecutionException, InterruptedException, IOException {
+        final AtomicReference<FlowFileRecord> flowFileReference = new AtomicReference<>();
+        final ProcessorNode createProcessor = createGenerateProcessor(1024 * 1024, flowFileReference);
+
+        connect(createProcessor, getTerminateProcessor(), REL_SUCCESS);
+        connect(createProcessor, getTerminateAllProcessor(), REL_SUCCESS);
+
+        triggerOnce(createProcessor);
+
+        final ContentClaim contentClaim = flowFileReference.get().getContentClaim();
+        assertEquals(2, getContentRepository().getClaimantCount(contentClaim));
+        assertTrue(contentClaim.getResourceClaim().isInUse());
+
+        triggerOnce(getTerminateProcessor());
+        assertEquals(1, getContentRepository().getClaimantCount(contentClaim));
+        assertTrue(contentClaim.getResourceClaim().isInUse());
+
+        triggerOnce(getTerminateAllProcessor());
+        assertEquals(0, getContentRepository().getClaimantCount(contentClaim));
+        assertFalse(contentClaim.getResourceClaim().isInUse());
+
+        assertEquals(0, ((WriteAheadFlowFileRepository) getFlowFileRepository()).checkpoint());
+
+        waitForClaimDestruction(contentClaim);
+    }
+
+
+    private void waitForClaimDestruction(final ContentClaim contentClaim) {
+        final Path path = ((FileSystemRepository) getContentRepository()).getPath(contentClaim, false);
+
+        while (Files.exists(path)) {
+            try {
+                Thread.sleep(10L);
+            } catch (final Exception e) {
+            }
+        }
+    }
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/lifecycle/FlowFileRepositoryLifecycleIT.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/lifecycle/FlowFileRepositoryLifecycleIT.java
new file mode 100644
index 0000000..257111b
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/lifecycle/FlowFileRepositoryLifecycleIT.java
@@ -0,0 +1,147 @@
+/*
+ * 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.integration.lifecycle;
+
+import org.apache.nifi.controller.ProcessorNode;
+import org.apache.nifi.controller.queue.DropFlowFileState;
+import org.apache.nifi.controller.queue.DropFlowFileStatus;
+import org.apache.nifi.controller.queue.FlowFileQueue;
+import org.apache.nifi.controller.repository.FlowFileRecord;
+import org.apache.nifi.controller.repository.FlowFileSwapManager;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.integration.FrameworkIntegrationTest;
+import org.apache.nifi.provenance.ProvenanceEventType;
+import org.apache.nifi.reporting.Bulletin;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.testng.Assert.assertNull;
+import static org.testng.AssertJUnit.assertNotNull;
+
+public class FlowFileRepositoryLifecycleIT extends FrameworkIntegrationTest {
+
+    @Test
+    public void testFlowFilesReloadedIntoQueuesUponRestart() throws ExecutionException, InterruptedException, IOException {
+        final ProcessorNode procNode = createProcessorNode((context, session) -> {
+            FlowFile flowFile = session.get();
+            if (flowFile == null) {
+                flowFile = session.create();
+            }
+
+            flowFile = session.putAttribute(flowFile,"creator", "unit-test");
+            session.transfer(flowFile, REL_SUCCESS);
+        }, REL_SUCCESS);
+
+        connect(procNode, getNopProcessor(), REL_SUCCESS);
+        start(procNode);
+        final FlowFileQueue queue = getDestinationQueue(procNode, REL_SUCCESS);
+
+        while (queue.isEmpty()) {
+            try {
+                Thread.sleep(10L);
+            } catch (InterruptedException e) {
+            }
+        }
+
+        stop(procNode).get();
+
+        final int queueSize = queue.size().getObjectCount();
+        assertTrue(queueSize > 0);
+        assertEquals(0L, queue.size().getByteCount());
+
+        assertProvenanceEventCount(ProvenanceEventType.CREATE, queueSize);
+
+        shutdown();
+
+        final FlowFileQueue restoredQueue = createFlowFileQueue(queue.getIdentifier());
+        initialize(() -> Collections.singleton(restoredQueue));
+
+        for (int i=0; i < queueSize; i++) {
+            final FlowFileRecord flowFileRecord = restoredQueue.poll(Collections.emptySet());
+            assertNotNull(flowFileRecord);
+            assertEquals("unit-test", flowFileRecord.getAttribute("creator"));
+        }
+
+        assertFalse(restoredQueue.isEmpty());
+        assertTrue(restoredQueue.isActiveQueueEmpty());
+        assertNull(restoredQueue.poll(Collections.emptySet()));
+    }
+
+
+    @Test
+    public void testMissingSwapFileOnSwapIn() throws ExecutionException, InterruptedException, IOException {
+        final ProcessorNode createProcessor = createProcessorNode((context, session) -> {
+            for (int i=0; i < 30_000; i++) {
+                session.transfer(session.create(), REL_SUCCESS);
+            }
+        }, REL_SUCCESS);
+
+        connect(createProcessor, getTerminateAllProcessor(), REL_SUCCESS);
+        triggerOnce(createProcessor);
+
+        // Verify queue sizes
+        final FlowFileQueue flowFileQueue = getDestinationQueue(createProcessor, REL_SUCCESS);
+        assertEquals(30_000, flowFileQueue.size().getObjectCount());
+        assertEquals(10_000, flowFileQueue.getQueueDiagnostics().getLocalQueuePartitionDiagnostics().getSwapQueueSize().getObjectCount());
+        assertEquals(1, flowFileQueue.getQueueDiagnostics().getLocalQueuePartitionDiagnostics().getSwapFileCount());
+        assertEquals(20_000, flowFileQueue.getQueueDiagnostics().getLocalQueuePartitionDiagnostics().getActiveQueueSize().getObjectCount());
+
+        // Delete all swap files
+        final FlowFileSwapManager swapManager = getFlowController().createSwapManager();
+        swapManager.purge();
+
+        // Verify queue sizes haven't changed
+        assertEquals(30_000, flowFileQueue.size().getObjectCount());
+        assertEquals(10_000, flowFileQueue.getQueueDiagnostics().getLocalQueuePartitionDiagnostics().getSwapQueueSize().getObjectCount());
+        assertEquals(1, flowFileQueue.getQueueDiagnostics().getLocalQueuePartitionDiagnostics().getSwapFileCount());
+        assertEquals(20_000, flowFileQueue.getQueueDiagnostics().getLocalQueuePartitionDiagnostics().getActiveQueueSize().getObjectCount());
+
+        for (int i=0; i < 5; i++) {
+            triggerOnce(getTerminateAllProcessor());
+
+            // Verify new queue sizes
+            assertEquals(10_000, flowFileQueue.size().getObjectCount());
+            assertEquals(10_000, flowFileQueue.getQueueDiagnostics().getLocalQueuePartitionDiagnostics().getSwapQueueSize().getObjectCount());
+            assertEquals(1, flowFileQueue.getQueueDiagnostics().getLocalQueuePartitionDiagnostics().getSwapFileCount());
+            assertEquals(0, flowFileQueue.getQueueDiagnostics().getLocalQueuePartitionDiagnostics().getActiveQueueSize().getObjectCount());
+
+            final List<Bulletin> bulletins = getFlowController().getBulletinRepository().findBulletinsForController();
+            assertEquals(1, bulletins.size());
+        }
+
+        assertProvenanceEventCount(ProvenanceEventType.CREATE, 30_000);
+        assertProvenanceEventCount(ProvenanceEventType.DROP, 20_000);
+
+        final DropFlowFileStatus status = flowFileQueue.dropFlowFiles("unit-test-id-1", "unit-test");
+        while (status.getState() != DropFlowFileState.COMPLETE) {
+            Thread.sleep(10L);
+        }
+
+        assertEquals(0, status.getDroppedSize().getObjectCount());
+        assertEquals(10_000, flowFileQueue.size().getObjectCount());
+        assertEquals(10_000, flowFileQueue.getQueueDiagnostics().getLocalQueuePartitionDiagnostics().getSwapQueueSize().getObjectCount());
+        assertEquals(1, flowFileQueue.getQueueDiagnostics().getLocalQueuePartitionDiagnostics().getSwapFileCount());
+        assertEquals(0, flowFileQueue.getQueueDiagnostics().getLocalQueuePartitionDiagnostics().getActiveQueueSize().getObjectCount());
+    }
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/processgroup/StandardProcessGroupIT.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/processgroup/StandardProcessGroupIT.java
new file mode 100644
index 0000000..15dc5a8
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/processgroup/StandardProcessGroupIT.java
@@ -0,0 +1,100 @@
+/*
+ * 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.integration.processgroup;
+
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.PropertyDescriptor.Builder;
+import org.apache.nifi.controller.ComponentNode;
+import org.apache.nifi.controller.ProcessorNode;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.groups.ProcessGroup;
+import org.apache.nifi.integration.FrameworkIntegrationTest;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class StandardProcessGroupIT extends FrameworkIntegrationTest {
+
+    @Test
+    public void testComponentsAffectedByVariableOverridden() {
+        final ProcessGroup child = getFlowController().getFlowManager().createProcessGroup("child");
+        child.setName("Child");
+        child.setVariables(Collections.singletonMap("number", "5"));
+
+        getRootGroup().setVariables(Collections.singletonMap("number", "1"));
+        getRootGroup().addProcessGroup(child);
+
+        final ProcessorNode processor = createProcessorNode(NumberRefProcessor.class);
+        processor.setProperties(Collections.singletonMap(NumberRefProcessor.NUMBER.getName(), "${number}"));
+        moveProcessor(processor, child);
+
+        final Set<ComponentNode> componentsAffected = child.getComponentsAffectedByVariable("number");
+        assertEquals(1, componentsAffected.size());
+        assertTrue(componentsAffected.contains(processor));
+
+        final Set<ComponentNode> rootAffected = getRootGroup().getComponentsAffectedByVariable("number");
+        assertTrue(rootAffected.isEmpty());
+
+        processor.setScheduldingPeriod("1 hour");
+        child.startProcessor(processor, false);
+
+        getRootGroup().setVariables(Collections.singletonMap("number", "2"));
+
+        try {
+            child.setVariables(Collections.singletonMap("number", "10"));
+            Assert.fail("Updated variable that is referenced by a running processor");
+        } catch (final IllegalStateException ise) {
+            // Expected
+        }
+
+        child.stopProcessor(processor);
+    }
+
+
+    public static class NumberRefProcessor extends AbstractProcessor {
+        static final PropertyDescriptor NUMBER = new Builder()
+            .name("Number")
+            .displayName("Number")
+            .description("A Number")
+            .required(true)
+            .addValidator(StandardValidators.NUMBER_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .defaultValue("1")
+            .build();
+
+        @Override
+        protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+            return Collections.singletonList(NUMBER);
+        }
+
+        @Override
+        public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+
+        }
+    }
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/processor/BiConsumerProcessor.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/processor/BiConsumerProcessor.java
new file mode 100644
index 0000000..2fe6029
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/processor/BiConsumerProcessor.java
@@ -0,0 +1,57 @@
+/*
+ * 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.integration.processor;
+
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+
+import java.util.Set;
+import java.util.function.BiConsumer;
+
+public class BiConsumerProcessor extends AbstractProcessor {
+    private BiConsumer<ProcessContext, ProcessSession> trigger;
+    private Set<Relationship> relationships;
+
+    public void setTrigger(final BiConsumer<ProcessContext, ProcessSession> trigger) {
+        this.trigger = trigger;
+    }
+
+    public void setRelationships(final Set<Relationship> relationships) {
+        this.relationships = relationships;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        if (relationships == null) {
+            throw new IllegalStateException("Relationships have not been initialized");
+        }
+
+        return relationships;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        if (trigger == null) {
+            throw new IllegalStateException("Trigger has not been initialized");
+        }
+
+        trigger.accept(context, session);
+    }
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/provenance/ProvenanceEventsIT.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/provenance/ProvenanceEventsIT.java
new file mode 100644
index 0000000..eb710fb
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/provenance/ProvenanceEventsIT.java
@@ -0,0 +1,423 @@
+/*
+ * 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.integration.provenance;
+
+import org.apache.nifi.controller.ProcessorNode;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.integration.FrameworkIntegrationTest;
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+import org.apache.nifi.provenance.ProvenanceEventRepository;
+import org.apache.nifi.provenance.ProvenanceEventType;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ExecutionException;
+
+import static junit.framework.TestCase.assertTrue;
+import static org.junit.Assert.assertEquals;
+import static org.testng.Assert.assertNull;
+
+public class ProvenanceEventsIT extends FrameworkIntegrationTest {
+
+    @Test
+    public void testCreateEventIfNewFlowFileWithoutReceive() throws ExecutionException, InterruptedException, IOException {
+        final ProcessorNode createProcessor = createProcessorNode((context, session) -> {
+            FlowFile flowFile = session.create();
+
+            final Map<String, String> attrs = new HashMap<>();
+            attrs.put("test", "integration");
+            attrs.put("integration", "true");
+
+            flowFile = session.putAllAttributes(flowFile, attrs);
+            session.transfer(flowFile, REL_SUCCESS);
+        }, REL_SUCCESS);
+
+        connect(createProcessor, getTerminateAllProcessor(), REL_SUCCESS);
+        triggerOnce(createProcessor);
+
+        // There should be exactly 1 event.
+        final ProvenanceEventRepository provRepo = getProvenanceRepository();
+        assertEquals(0L, provRepo.getMaxEventId().longValue());
+
+        final ProvenanceEventRecord firstEvent = provRepo.getEvent(0L);
+        assertEquals(ProvenanceEventType.CREATE, firstEvent.getEventType());
+        assertEquals("integration", firstEvent.getAttribute("test"));
+        assertEquals("true", firstEvent.getAttribute("integration"));
+    }
+
+    @Test
+    public void testNoCreateEventIfReceiveExplicitlyCreated() throws ExecutionException, InterruptedException, IOException {
+        final ProcessorNode createProcessor = createProcessorNode((context, session) -> {
+            FlowFile flowFile = session.create();
+
+            final Map<String, String> attrs = new HashMap<>();
+            attrs.put("test", "integration");
+            attrs.put("integration", "true");
+
+            flowFile = session.putAllAttributes(flowFile, attrs);
+            session.getProvenanceReporter().receive(flowFile, "nifi://unit.test");
+            session.transfer(flowFile, REL_SUCCESS);
+        }, REL_SUCCESS);
+
+        connect(createProcessor, getTerminateAllProcessor(), REL_SUCCESS);
+        triggerOnce(createProcessor);
+
+        // There should be exactly 1 event.
+        final ProvenanceEventRepository provRepo = getProvenanceRepository();
+        assertEquals(0L, provRepo.getMaxEventId().longValue());
+
+        final ProvenanceEventRecord firstEvent = provRepo.getEvent(0L);
+        assertEquals(ProvenanceEventType.RECEIVE, firstEvent.getEventType());
+        assertEquals("integration", firstEvent.getAttribute("test"));
+        assertEquals("true", firstEvent.getAttribute("integration"));
+        assertEquals("nifi://unit.test", firstEvent.getTransitUri());
+    }
+
+    @Test
+    public void testDropEventIfRoutedToAutoTerminatedRelationship() throws ExecutionException, InterruptedException, IOException {
+        final ProcessorNode createProcessor = createProcessorNode((context, session) -> {
+            FlowFile flowFile = session.create();
+
+            final Map<String, String> attrs = new HashMap<>();
+            attrs.put("test", "integration");
+            attrs.put("integration", "true");
+
+            flowFile = session.putAllAttributes(flowFile, attrs);
+            session.transfer(flowFile, REL_SUCCESS);
+        }, REL_SUCCESS);
+
+        createProcessor.setAutoTerminatedRelationships(Collections.singleton(REL_SUCCESS));
+
+        triggerOnce(createProcessor);
+
+        // There should be exactly 1 event.
+        final ProvenanceEventRepository provRepo = getProvenanceRepository();
+        assertEquals(1L, provRepo.getMaxEventId().longValue());
+
+        final ProvenanceEventRecord firstEvent = provRepo.getEvent(0L);
+        assertEquals(ProvenanceEventType.CREATE, firstEvent.getEventType());
+        assertEquals("integration", firstEvent.getAttribute("test"));
+        assertEquals("true", firstEvent.getAttribute("integration"));
+
+        final ProvenanceEventRecord secondEvent = provRepo.getEvent(1L);
+        assertEquals(ProvenanceEventType.DROP, secondEvent.getEventType());
+        assertEquals("integration", secondEvent.getAttribute("test"));
+        assertEquals("true", secondEvent.getAttribute("integration"));
+    }
+
+    @Test
+    public void testNoEventsIfExplicitlyRemoved() throws ExecutionException, InterruptedException, IOException {
+        final ProcessorNode createProcessor = createProcessorNode((context, session) -> {
+            FlowFile flowFile = session.create();
+
+            final Map<String, String> attrs = new HashMap<>();
+            attrs.put("test", "integration");
+            attrs.put("integration", "true");
+
+            flowFile = session.putAllAttributes(flowFile, attrs);
+            session.remove(flowFile);
+        }, REL_SUCCESS);
+
+        createProcessor.setAutoTerminatedRelationships(Collections.singleton(REL_SUCCESS));
+
+        triggerOnce(createProcessor);
+
+        // There should be exactly 1 event.
+        final ProvenanceEventRepository provRepo = getProvenanceRepository();
+        assertEquals(-1L, provRepo.getMaxEventId().longValue());
+        assertTrue(provRepo.getEvents(0L, 1000).isEmpty());
+    }
+
+    @Test
+    public void testAttributesModifiedIfNothingElse() throws ExecutionException, InterruptedException, IOException {
+        final ProcessorNode createProcessor = createGenerateProcessor(0);
+
+        final ProcessorNode updateProcessor = createProcessorNode((context, session) -> {
+            FlowFile flowFile = session.get();
+
+            final Map<String, String> attrs = new HashMap<>();
+            attrs.put("test", "integration");
+            attrs.put("integration", "true");
+
+            flowFile = session.putAllAttributes(flowFile, attrs);
+            session.transfer(flowFile, REL_SUCCESS);
+        }, REL_SUCCESS);
+
+        connect(createProcessor, updateProcessor, REL_SUCCESS);
+        connect(updateProcessor, getTerminateAllProcessor(), REL_SUCCESS);
+
+        triggerOnce(createProcessor);
+        triggerOnce(updateProcessor);
+
+        // There should be exactly 1 event.
+        final ProvenanceEventRepository provRepo = getProvenanceRepository();
+        assertEquals(1L, provRepo.getMaxEventId().longValue());
+
+        final ProvenanceEventRecord firstEvent = provRepo.getEvent(0L);
+        assertEquals(ProvenanceEventType.CREATE, firstEvent.getEventType());
+        assertNull(firstEvent.getAttribute("test"));
+        assertNull(firstEvent.getAttribute("integration"));
+
+        final ProvenanceEventRecord secondEvent = provRepo.getEvent(1L);
+        assertEquals(ProvenanceEventType.ATTRIBUTES_MODIFIED, secondEvent.getEventType());
+        assertEquals("integration", secondEvent.getAttribute("test"));
+        assertEquals("true", secondEvent.getAttribute("integration"));
+    }
+
+    @Test
+    public void testAttributesModifiedNotCreatedIfProcessorEmitsIt() throws ExecutionException, InterruptedException, IOException {
+        final ProcessorNode createProcessor = createGenerateProcessor(0);
+
+        final ProcessorNode updateProcessor = createProcessorNode((context, session) -> {
+            FlowFile flowFile = session.get();
+
+            final Map<String, String> attrs = new HashMap<>();
+            attrs.put("test", "integration");
+            attrs.put("integration", "true");
+
+            flowFile = session.putAllAttributes(flowFile, attrs);
+            session.getProvenanceReporter().modifyAttributes(flowFile, "Unit Test Details");
+            session.transfer(flowFile, REL_SUCCESS);
+        }, REL_SUCCESS);
+
+        connect(createProcessor, updateProcessor, REL_SUCCESS);
+        connect(updateProcessor, getTerminateAllProcessor(), REL_SUCCESS);
+
+        triggerOnce(createProcessor);
+        triggerOnce(updateProcessor);
+
+        // There should be exactly 1 event.
+        final ProvenanceEventRepository provRepo = getProvenanceRepository();
+        assertEquals(1L, provRepo.getMaxEventId().longValue());
+
+        final ProvenanceEventRecord firstEvent = provRepo.getEvent(0L);
+        assertEquals(ProvenanceEventType.CREATE, firstEvent.getEventType());
+        assertNull(firstEvent.getAttribute("test"));
+        assertNull(firstEvent.getAttribute("integration"));
+
+        final ProvenanceEventRecord secondEvent = provRepo.getEvent(1L);
+        assertEquals(ProvenanceEventType.ATTRIBUTES_MODIFIED, secondEvent.getEventType());
+        assertEquals("integration", secondEvent.getAttribute("test"));
+        assertEquals("true", secondEvent.getAttribute("integration"));
+        assertEquals("Unit Test Details", secondEvent.getDetails());
+    }
+
+    @Test
+    public void testAttributesModifiedNotCreatedIfProcessorEmitsOtherEvent() throws ExecutionException, InterruptedException, IOException {
+        final ProcessorNode createProcessor = createGenerateProcessor(0);
+
+        final ProcessorNode updateProcessor = createProcessorNode((context, session) -> {
+            FlowFile flowFile = session.get();
+
+            final Map<String, String> attrs = new HashMap<>();
+            attrs.put("test", "integration");
+            attrs.put("integration", "true");
+
+            flowFile = session.putAllAttributes(flowFile, attrs);
+            session.getProvenanceReporter().fetch(flowFile, "nifi://unit.test");
+            session.transfer(flowFile, REL_SUCCESS);
+        }, REL_SUCCESS);
+
+        connect(createProcessor, updateProcessor, REL_SUCCESS);
+        connect(updateProcessor, getTerminateAllProcessor(), REL_SUCCESS);
+
+        triggerOnce(createProcessor);
+        triggerOnce(updateProcessor);
+
+        // There should be exactly 1 event.
+        final ProvenanceEventRepository provRepo = getProvenanceRepository();
+        assertEquals(1L, provRepo.getMaxEventId().longValue());
+
+        final ProvenanceEventRecord firstEvent = provRepo.getEvent(0L);
+        assertEquals(ProvenanceEventType.CREATE, firstEvent.getEventType());
+        assertNull(firstEvent.getAttribute("test"));
+        assertNull(firstEvent.getAttribute("integration"));
+
+        final ProvenanceEventRecord secondEvent = provRepo.getEvent(1L);
+        assertEquals(ProvenanceEventType.FETCH, secondEvent.getEventType());
+        assertEquals("integration", secondEvent.getAttribute("test"));
+        assertEquals("true", secondEvent.getAttribute("integration"));
+        assertEquals("nifi://unit.test", secondEvent.getTransitUri());
+    }
+
+
+    @Test
+    public void testAttributesModifiedNotCreatedIfContentModified() throws ExecutionException, InterruptedException, IOException {
+        final ProcessorNode createProcessor = createGenerateProcessor(0);
+
+        final ProcessorNode updateProcessor = createProcessorNode((context, session) -> {
+            FlowFile flowFile = session.get();
+
+            final Map<String, String> attrs = new HashMap<>();
+            attrs.put("test", "integration");
+            attrs.put("integration", "true");
+
+            flowFile = session.putAllAttributes(flowFile, attrs);
+            flowFile = session.write(flowFile, out -> out.write('A'));
+            session.transfer(flowFile, REL_SUCCESS);
+        }, REL_SUCCESS);
+
+        connect(createProcessor, updateProcessor, REL_SUCCESS);
+        connect(updateProcessor, getTerminateAllProcessor(), REL_SUCCESS);
+
+        triggerOnce(createProcessor);
+        triggerOnce(updateProcessor);
+
+        // There should be exactly 1 event.
+        final ProvenanceEventRepository provRepo = getProvenanceRepository();
+        assertEquals(1L, provRepo.getMaxEventId().longValue());
+
+        final ProvenanceEventRecord firstEvent = provRepo.getEvent(0L);
+        assertEquals(ProvenanceEventType.CREATE, firstEvent.getEventType());
+        assertNull(firstEvent.getAttribute("test"));
+        assertNull(firstEvent.getAttribute("integration"));
+
+        final ProvenanceEventRecord secondEvent = provRepo.getEvent(1L);
+        assertEquals(ProvenanceEventType.CONTENT_MODIFIED, secondEvent.getEventType());
+        assertEquals("integration", secondEvent.getAttribute("test"));
+        assertEquals("true", secondEvent.getAttribute("integration"));
+    }
+
+
+    @Test
+    public void testNoAttributesModifiedOnJoin() throws ExecutionException, InterruptedException, IOException {
+        testJoin(false);
+    }
+
+    @Test
+    public void testNoAttributesModifiedOnJoinWithExplicitJoinEvent() throws ExecutionException, InterruptedException, IOException {
+        testJoin(true);
+    }
+
+    private void testJoin(final boolean emitJoinEventExplicitly) throws ExecutionException, InterruptedException, IOException {
+        final ProcessorNode createProcessor = createGenerateProcessor(0);
+
+        final ProcessorNode joinProcessor = createProcessorNode((context, session) -> {
+            final List<FlowFile> originals = new ArrayList<>();
+            FlowFile flowFile;
+            while ((flowFile = session.get()) != null) {
+                originals.add(flowFile);
+            }
+
+            FlowFile merged = session.create(originals);
+
+            final Map<String, String> attrs = new HashMap<>();
+            attrs.put("test", "integration");
+            attrs.put("integration", "true");
+
+            merged = session.putAllAttributes(merged, attrs);
+            merged = session.write(merged, out -> out.write('A'));
+
+            if (emitJoinEventExplicitly) {
+                session.getProvenanceReporter().join(originals, merged);
+            }
+
+            session.remove(originals);
+            session.transfer(merged, REL_SUCCESS);
+            session.getProvenanceReporter().route(merged, REL_SUCCESS);
+        }, REL_SUCCESS);
+
+        connect(createProcessor, joinProcessor, REL_SUCCESS);
+        joinProcessor.setAutoTerminatedRelationships(Collections.singleton(REL_SUCCESS));
+
+        for (int i=0; i < 3; i++) {
+            triggerOnce(createProcessor);
+        }
+
+        triggerOnce(joinProcessor);
+
+        final ProvenanceEventRepository provRepo = getProvenanceRepository();
+        assertEquals(8L, provRepo.getMaxEventId().longValue());
+
+        // Crete events are from the first 'generate' processor.
+        for (int i=0; i < 3; i++) {
+            assertEquals(ProvenanceEventType.CREATE, provRepo.getEvent(i).getEventType());
+        }
+
+        // Any FORK/JOIN events will occur first in the Process Session to ensure that any other events that reference the FlowFile
+        // that is created as a result have a FlowFile to actually reference.
+        final ProvenanceEventRecord joinEvent = provRepo.getEvent(3);
+        assertEquals(ProvenanceEventType.JOIN, joinEvent.getEventType());
+        assertEquals("integration", joinEvent.getAttribute("test"));
+        assertEquals("true", joinEvent.getAttribute("integration"));
+        assertEquals(3, joinEvent.getParentUuids().size());
+        assertEquals(1, joinEvent.getChildUuids().size());
+        assertEquals(joinEvent.getFlowFileUuid(), joinEvent.getChildUuids().get(0));
+
+        // Next event to occur in the Processor is the DORP event
+        for (int i=4; i < 7; i++) {
+            assertEquals(ProvenanceEventType.DROP, provRepo.getEvent(i).getEventType());
+        }
+
+        // Finally Processor will ROUTE the FlowFile
+        final ProvenanceEventRecord routeEvent = provRepo.getEvent(7);
+        assertEquals(ProvenanceEventType.ROUTE, routeEvent.getEventType());
+        assertEquals("success", routeEvent.getRelationship());
+        assertEquals("integration", routeEvent.getAttribute("test"));
+        assertEquals("true", routeEvent.getAttribute("integration"));
+
+        // Merged FlowFile is then auto-terminated.
+        final ProvenanceEventRecord dropJoinedEvent = provRepo.getEvent(8);
+        assertEquals(ProvenanceEventType.DROP, dropJoinedEvent.getEventType());
+        assertEquals("integration", dropJoinedEvent.getAttribute("test"));
+        assertEquals("true", dropJoinedEvent.getAttribute("integration"));
+    }
+
+    @Test
+    public void testForkAutoGenerated() throws ExecutionException, InterruptedException, IOException {
+        final ProcessorNode generateProcessor = createGenerateProcessor(0);
+        final ProcessorNode forkProcessor = createProcessorNode((context, session) -> {
+            FlowFile original = session.get();
+
+            for (int i=0; i < 3; i++) {
+                FlowFile child = session.create(original);
+                child = session.putAttribute(child, "i", String.valueOf(i));
+                session.transfer(child, REL_SUCCESS);
+            }
+
+            session.remove(original);
+        }, REL_SUCCESS);
+
+        connect(generateProcessor, forkProcessor, REL_SUCCESS);
+        connect(forkProcessor, getTerminateAllProcessor(), REL_SUCCESS);
+
+        triggerOnce(generateProcessor);
+        triggerOnce(forkProcessor);
+
+        final ProvenanceEventRepository provRepo = getProvenanceRepository();
+        assertEquals(2L, provRepo.getMaxEventId().longValue());
+
+        final ProvenanceEventRecord firstEvent = provRepo.getEvent(0L);
+        assertEquals(ProvenanceEventType.CREATE, firstEvent.getEventType());
+        assertNull(firstEvent.getAttribute("test"));
+        assertNull(firstEvent.getAttribute("integration"));
+
+        final ProvenanceEventRecord secondEvent = provRepo.getEvent(1L);
+        assertEquals(ProvenanceEventType.FORK, secondEvent.getEventType());
+        assertEquals(1, secondEvent.getParentUuids().size());
+        assertEquals(3, secondEvent.getChildUuids().size());
+        assertEquals(secondEvent.getFlowFileUuid(), secondEvent.getParentUuids().get(0));
+
+        final ProvenanceEventRecord thirdEvent = provRepo.getEvent(2L);
+        assertEquals(ProvenanceEventType.DROP, thirdEvent.getEventType());
+    }
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/versioned/ImportFlowIT.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/versioned/ImportFlowIT.java
new file mode 100644
index 0000000..9320ce5
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/versioned/ImportFlowIT.java
@@ -0,0 +1,163 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.integration.versioned;
+
+import org.apache.nifi.bundle.BundleCoordinate;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.components.validation.ValidationStatus;
+import org.apache.nifi.controller.ControllerService;
+import org.apache.nifi.controller.ProcessorNode;
+import org.apache.nifi.controller.service.ControllerServiceNode;
+import org.apache.nifi.groups.ProcessGroup;
+import org.apache.nifi.integration.DirectInjectionExtensionManager;
+import org.apache.nifi.integration.FrameworkIntegrationTest;
+import org.apache.nifi.integration.cs.LongValidatingControllerService;
+import org.apache.nifi.integration.cs.NopServiceReferencingProcessor;
+import org.apache.nifi.processor.Processor;
+import org.apache.nifi.registry.bucket.Bucket;
+import org.apache.nifi.registry.flow.Bundle;
+import org.apache.nifi.registry.flow.VersionedControllerService;
+import org.apache.nifi.registry.flow.VersionedFlow;
+import org.apache.nifi.registry.flow.VersionedFlowSnapshot;
+import org.apache.nifi.registry.flow.VersionedFlowSnapshotMetadata;
+import org.apache.nifi.registry.flow.VersionedProcessGroup;
+import org.apache.nifi.registry.flow.VersionedProcessor;
+import org.apache.nifi.registry.flow.mapping.NiFiRegistryFlowMapper;
+import org.junit.Test;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
+
+import static junit.framework.TestCase.assertTrue;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+
+public class ImportFlowIT extends FrameworkIntegrationTest {
+
+    @Override
+    protected void injectExtensionTypes(final DirectInjectionExtensionManager extensionManager) {
+        extensionManager.injectExtensionType(Processor.class, NopServiceReferencingProcessor.class);
+        extensionManager.injectExtensionType(ControllerService.class, LongValidatingControllerService.class);
+    }
+
+    @Test
+    public void testImportFlowWithProcessorAndControllerService() throws ExecutionException, InterruptedException {
+        // Build a Versioned Flow that consists of a Controller Service and a Processor
+        // that references that Controller Service.
+        final ControllerServiceNode controllerService = createControllerServiceNode(LongValidatingControllerService.class);
+        controllerService.setProperties(Collections.singletonMap(LongValidatingControllerService.DELAY.getName(), "250 millis"));
+
+        final ProcessorNode processor = createProcessorNode(NopServiceReferencingProcessor.class);
+        processor.setAutoTerminatedRelationships(Collections.singleton(REL_SUCCESS));
+        processor.setProperties(Collections.singletonMap(NopServiceReferencingProcessor.SERVICE.getName(), controllerService.getIdentifier()));
+
+        final VersionedFlowSnapshot proposedFlow = createFlowSnapshot(Collections.singletonList(controllerService), Collections.singletonList(processor));
+
+        // Create an Inner Process Group and update it to match the Versioned Flow.
+        final ProcessGroup innerGroup = getFlowController().getFlowManager().createProcessGroup("inner-group-id");
+        innerGroup.setName("Inner Group");
+        getRootGroup().addProcessGroup(innerGroup);
+
+        innerGroup.updateFlow(proposedFlow, null, false, true, false);
+
+        // Ensure that the controller service is valid and enable it.
+        final Set<ControllerServiceNode> serviceNodes = innerGroup.findAllControllerServices();
+        assertEquals(1, serviceNodes.size());
+
+        final ControllerServiceNode serviceNode = serviceNodes.iterator().next();
+        final ValidationStatus validationStatus = serviceNode.performValidation();
+        assertEquals(ValidationStatus.VALID, validationStatus);
+        getFlowController().getControllerServiceProvider().enableControllerService(serviceNode).get();
+        assertTrue(serviceNode.isActive());
+
+        // Ensure that the processor is valid.
+        final List<ProcessorNode> processorNodes = innerGroup.findAllProcessors();
+        assertEquals(1, processorNodes.size());
+
+        final ProcessorNode procNode = processorNodes.get(0);
+        final ValidationStatus procValidationStatus = procNode.performValidation();
+        final Collection<ValidationResult> validationErrors = procNode.getValidationErrors();
+        System.out.println(validationErrors);
+        assertEquals(Collections.emptyList(), validationErrors);
+        assertEquals(ValidationStatus.VALID, procValidationStatus);
+
+        // Ensure that the reference to the controller service was properly updated
+        final String referencedServiceId = procNode.getProperty(NopServiceReferencingProcessor.SERVICE);
+        assertEquals(serviceNode.getIdentifier(), referencedServiceId);
+        assertNotEquals("service-id", referencedServiceId);
+    }
+
+
+    private VersionedFlowSnapshot createFlowSnapshot(final List<ControllerServiceNode> controllerServices, final List<ProcessorNode> processors) {
+        final VersionedFlowSnapshotMetadata snapshotMetadata = new VersionedFlowSnapshotMetadata();
+        snapshotMetadata.setAuthor("unit-test");
+        snapshotMetadata.setBucketIdentifier("unit-test-bucket");
+        snapshotMetadata.setFlowIdentifier("unit-test-flow");
+        snapshotMetadata.setTimestamp(System.currentTimeMillis());
+        snapshotMetadata.setVersion(1);
+
+        final Bucket bucket = new Bucket();
+        bucket.setCreatedTimestamp(System.currentTimeMillis());
+        bucket.setIdentifier("unit-test-bucket");
+        bucket.setName("Unit Test Bucket");
+
+        final VersionedFlow flow = new VersionedFlow();
+        flow.setBucketIdentifier("unit-test-bucket");
+        flow.setBucketName("Unit Test Bucket");
+        flow.setCreatedTimestamp(System.currentTimeMillis());
+        flow.setIdentifier("unit-test-flow");
+        flow.setName("Unit Test Flow");
+
+        final BundleCoordinate coordinate = getSystemBundle().getBundleDetails().getCoordinate();
+        final Bundle bundle = new Bundle();
+        bundle.setArtifact(coordinate.getId());
+        bundle.setGroup(coordinate.getGroup());
+        bundle.setVersion(coordinate.getVersion());
+
+        final NiFiRegistryFlowMapper flowMapper = new NiFiRegistryFlowMapper(getExtensionManager());
+
+        final Set<VersionedProcessor> versionedProcessors = new HashSet<>();
+        for (final ProcessorNode processor : processors) {
+            final VersionedProcessor versionedProcessor = flowMapper.mapProcessor(processor, getFlowController().getControllerServiceProvider());
+            versionedProcessors.add(versionedProcessor);
+        }
+
+        final Set<VersionedControllerService> services = new HashSet<>();
+        for (final ControllerServiceNode serviceNode : controllerServices) {
+            final VersionedControllerService service = flowMapper.mapControllerService(serviceNode, getFlowController().getControllerServiceProvider());
+            services.add(service);
+        }
+
+        final VersionedProcessGroup flowContents = new VersionedProcessGroup();
+        flowContents.setIdentifier("unit-test-flow-contents");
+        flowContents.setName("Unit Test");
+        flowContents.setProcessors(versionedProcessors);
+        flowContents.setControllerServices(services);
+
+        final VersionedFlowSnapshot versionedFlowSnapshot = new VersionedFlowSnapshot();
+        versionedFlowSnapshot.setSnapshotMetadata(snapshotMetadata);
+        versionedFlowSnapshot.setBucket(bucket);
+        versionedFlowSnapshot.setFlow(flow);
+        versionedFlowSnapshot.setFlowContents(flowContents);
+
+        return versionedFlowSnapshot;
+    }
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/int-tests/default-nifi.properties b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/int-tests/default-nifi.properties
new file mode 100644
index 0000000..1bbb49a
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/int-tests/default-nifi.properties
@@ -0,0 +1,256 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+# Core Properties #
+nifi.flow.configuration.file=./target/conf/flow.xml.gz
+nifi.flow.configuration.archive.enabled=true
+nifi.flow.configuration.archive.dir=./target/conf/archive/
+nifi.flow.configuration.archive.max.time=30 days
+nifi.flow.configuration.archive.max.storage=500 MB
+nifi.flow.configuration.archive.max.count=
+nifi.flowcontroller.autoResumeState=true
+nifi.flowcontroller.graceful.shutdown.period=10 sec
+nifi.flowservice.writedelay.interval=500 ms
+nifi.administrative.yield.duration=100 millis
+# If a component has no work to do (is "bored"), how long should we wait before checking again for work?
+nifi.bored.yield.duration=10 millis
+nifi.queue.backpressure.count=10000
+nifi.queue.backpressure.size=1 GB
+
+nifi.authorizer.configuration.file=./target/conf/authorizers.xml
+nifi.login.identity.provider.configuration.file=./target/conf/login-identity-providers.xml
+nifi.templates.directory=./target/conf/templates
+nifi.ui.banner.text=
+nifi.ui.autorefresh.interval=30 sec
+nifi.nar.library.directory=./target/lib
+nifi.nar.library.autoload.directory=./target/extensions
+nifi.nar.working.directory=./target/work/nar/
+nifi.documentation.working.directory=./target/work/docs/components
+
+####################
+# State Management #
+####################
+nifi.state.management.configuration.file=src/test/resources/int-tests/state-management.xml
+# The ID of the local state provider
+nifi.state.management.provider.local=local-provider
+# The ID of the cluster-wide state provider. This will be ignored if NiFi is not clustered but must be populated if running in a cluster.
+nifi.state.management.provider.cluster=zk-provider
+# Specifies whether or not this instance of NiFi should run an embedded ZooKeeper server
+nifi.state.management.embedded.zookeeper.start=false
+# Properties file that provides the ZooKeeper properties to use if <nifi.state.management.embedded.zookeeper.start> is set to true
+nifi.state.management.embedded.zookeeper.properties=./target/conf/zookeeper.properties
+
+
+# H2 Settings
+nifi.database.directory=./target/database_repository
+nifi.h2.url.append=;LOCK_TIMEOUT=25000;WRITE_DELAY=0;AUTO_SERVER=FALSE
+
+# FlowFile Repository
+nifi.flowfile.repository.implementation=org.apache.nifi.controller.repository.WriteAheadFlowFileRepository
+nifi.flowfile.repository.wal.implementation=org.apache.nifi.wali.SequentialAccessWriteAheadLog
+nifi.flowfile.repository.directory=./target/int-tests/flowfile_repository
+nifi.flowfile.repository.partitions=256
+nifi.flowfile.repository.checkpoint.interval=5 mins
+nifi.flowfile.repository.always.sync=false
+
+nifi.swap.manager.implementation=org.apache.nifi.controller.FileSystemSwapManager
+nifi.queue.swap.threshold=20000
+nifi.swap.in.period=5 sec
+nifi.swap.in.threads=1
+nifi.swap.out.period=5 sec
+nifi.swap.out.threads=4
+
+# Content Repository
+nifi.content.repository.implementation=org.apache.nifi.controller.repository.FileSystemRepository
+nifi.content.claim.max.appendable.size=1 MB
+nifi.content.claim.max.flow.files=100
+nifi.content.repository.directory.default=./target/int-tests/content_repository
+nifi.content.repository.archive.max.retention.period=12 hours
+nifi.content.repository.archive.max.usage.percentage=50%
+nifi.content.repository.archive.enabled=true
+nifi.content.repository.always.sync=false
+nifi.content.viewer.url=../nifi-content-viewer/
+
+# Provenance Repository Properties
+nifi.provenance.repository.implementation=org.apache.nifi.provenance.WriteAheadProvenanceRepository
+nifi.provenance.repository.debug.frequency=1_000_000
+nifi.provenance.repository.encryption.key.provider.implementation=
+nifi.provenance.repository.encryption.key.provider.location=
+nifi.provenance.repository.encryption.key.id=
+nifi.provenance.repository.encryption.key=
+
+# Persistent Provenance Repository Properties
+nifi.provenance.repository.directory.default=./target/int-tests/provenance_repository
+nifi.provenance.repository.max.storage.time=24 hours
+nifi.provenance.repository.max.storage.size=1 GB
+nifi.provenance.repository.rollover.time=30 secs
+nifi.provenance.repository.rollover.size=100 MB
+nifi.provenance.repository.query.threads=2
+nifi.provenance.repository.index.threads=2
+nifi.provenance.repository.compress.on.rollover=true
+nifi.provenance.repository.always.sync=false
+# Comma-separated list of fields. Fields that are not indexed will not be searchable. Valid fields are:
+# EventType, FlowFileUUID, Filename, TransitURI, ProcessorID, AlternateIdentifierURI, Relationship, Details
+nifi.provenance.repository.indexed.fields=EventType, FlowFileUUID, Filename, ProcessorID, Relationship
+# FlowFile Attributes that should be indexed and made searchable.  Some examples to consider are filename, uuid, mime.type
+nifi.provenance.repository.indexed.attributes=
+# Large values for the shard size will result in more Java heap usage when searching the Provenance Repository
+# but should provide better performance
+nifi.provenance.repository.index.shard.size=500 MB
+# Indicates the maximum length that a FlowFile attribute can be when retrieving a Provenance Event from
+# the repository. If the length of any attribute exceeds this value, it will be truncated when the event is retrieved.
+nifi.provenance.repository.max.attribute.length=65536
+nifi.provenance.repository.concurrent.merge.threads=2
+
+
+# Volatile Provenance Respository Properties
+nifi.provenance.repository.buffer.size=100000
+
+# Component Status Repository
+nifi.components.status.repository.implementation=org.apache.nifi.controller.status.history.VolatileComponentStatusRepository
+nifi.components.status.repository.buffer.size=1440
+nifi.components.status.snapshot.frequency=1 min
+
+# Site to Site properties
+nifi.remote.input.host=
+nifi.remote.input.secure=false
+nifi.remote.input.socket.port=
+nifi.remote.input.http.enabled=true
+nifi.remote.input.http.transaction.ttl=30 sec
+nifi.remote.contents.cache.expiration=30 secs
+
+# web properties #
+nifi.web.war.directory=./target/lib
+nifi.web.http.host=
+nifi.web.http.port=8080
+nifi.web.http.network.interface.default=
+nifi.web.https.host=
+nifi.web.https.port=
+nifi.web.https.network.interface.default=
+nifi.web.jetty.working.directory=./target/work/jetty
+nifi.web.jetty.threads=200
+nifi.web.max.header.size=16 KB
+nifi.web.proxy.context.path=
+nifi.web.proxy.host=
+
+# security properties #
+nifi.sensitive.props.key=
+nifi.sensitive.props.key.protected=
+nifi.sensitive.props.algorithm=PBEWITHMD5AND256BITAES-CBC-OPENSSL
+nifi.sensitive.props.provider=BC
+nifi.sensitive.props.additional.keys=
+
+nifi.security.keystore=
+nifi.security.keystoreType=
+nifi.security.keystorePasswd=
+nifi.security.keyPasswd=
+nifi.security.truststore=
+nifi.security.truststoreType=
+nifi.security.truststorePasswd=
+nifi.security.user.authorizer=managed-authorizer
+nifi.security.user.login.identity.provider=
+nifi.security.ocsp.responder.url=
+nifi.security.ocsp.responder.certificate=
+
+# OpenId Connect SSO Properties #
+nifi.security.user.oidc.discovery.url=
+nifi.security.user.oidc.connect.timeout=5 secs
+nifi.security.user.oidc.read.timeout=5 secs
+nifi.security.user.oidc.client.id=
+nifi.security.user.oidc.client.secret=
+nifi.security.user.oidc.preferred.jwsalgorithm=
+
+# Apache Knox SSO Properties #
+nifi.security.user.knox.url=
+nifi.security.user.knox.publicKey=
+nifi.security.user.knox.cookieName=hadoop-jwt
+nifi.security.user.knox.audiences=
+
+# Identity Mapping Properties #
+# These properties allow normalizing user identities such that identities coming from different identity providers
+# (certificates, LDAP, Kerberos) can be treated the same internally in NiFi. The following example demonstrates normalizing
+# DNs from certificates and principals from Kerberos into a common identity string:
+#
+# nifi.security.identity.mapping.pattern.dn=^CN=(.*?), OU=(.*?), O=(.*?), L=(.*?), ST=(.*?), C=(.*?)$
+# nifi.security.identity.mapping.value.dn=$1@$2
+# nifi.security.identity.mapping.transform.dn=NONE
+# nifi.security.identity.mapping.pattern.kerb=^(.*?)/instance@(.*?)$
+# nifi.security.identity.mapping.value.kerb=$1@$2
+# nifi.security.identity.mapping.transform.kerb=UPPER
+
+# Group Mapping Properties #
+# These properties allow normalizing group names coming from external sources like LDAP. The following example
+# lowercases any group name.
+#
+# nifi.security.group.mapping.pattern.anygroup=^(.*)$
+# nifi.security.group.mapping.value.anygroup=$1
+# nifi.security.group.mapping.transform.anygroup=LOWER
+
+# cluster common properties (all nodes must have same values) #
+nifi.cluster.protocol.heartbeat.interval=5 sec
+nifi.cluster.protocol.is.secure=false
+
+# cluster node properties (only configure for cluster nodes) #
+nifi.cluster.is.node=false
+nifi.cluster.node.address=
+nifi.cluster.node.protocol.port=
+nifi.cluster.node.protocol.threads=10
+nifi.cluster.node.protocol.max.threads=50
+nifi.cluster.node.event.history.size=25
+nifi.cluster.node.connection.timeout=5 sec
+nifi.cluster.node.read.timeout=5 sec
+nifi.cluster.node.max.concurrent.requests=100
+nifi.cluster.firewall.file=
+nifi.cluster.flow.election.max.wait.time=5 mins
+nifi.cluster.flow.election.max.candidates=
+
+# cluster load balancing properties #
+nifi.cluster.load.balance.host=
+nifi.cluster.load.balance.port=6342
+nifi.cluster.load.balance.connections.per.node=4
+nifi.cluster.load.balance.max.thread.count=8
+nifi.cluster.load.balance.comms.timeout=30 sec
+
+# zookeeper properties, used for cluster management #
+nifi.zookeeper.connect.string=
+nifi.zookeeper.connect.timeout=3 secs
+nifi.zookeeper.session.timeout=3 secs
+nifi.zookeeper.root.node=/nifi
+
+# Zookeeper properties for the authentication scheme used when creating acls on znodes used for cluster management
+# Values supported for nifi.zookeeper.auth.type are "default", which will apply world/anyone rights on znodes
+# and "sasl" which will give rights to the sasl/kerberos identity used to authenticate the nifi node
+# The identity is determined using the value in nifi.kerberos.service.principal and the removeHostFromPrincipal
+# and removeRealmFromPrincipal values (which should align with the kerberos.removeHostFromPrincipal and kerberos.removeRealmFromPrincipal
+# values configured on the zookeeper server).
+nifi.zookeeper.auth.type=
+nifi.zookeeper.kerberos.removeHostFromPrincipal=
+nifi.zookeeper.kerberos.removeRealmFromPrincipal=
+
+# kerberos #
+nifi.kerberos.krb5.file=
+
+# kerberos service principal #
+nifi.kerberos.service.principal=
+nifi.kerberos.service.keytab.location=
+
+# kerberos spnego principal #
+nifi.kerberos.spnego.principal=
+nifi.kerberos.spnego.keytab.location=
+nifi.kerberos.spnego.authentication.expiration=12 hours
+
+# external properties files for variable registry
+# supports a comma delimited list of file locations
+nifi.variable.registry.properties=
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/int-tests/state-management.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/int-tests/state-management.xml
new file mode 100644
index 0000000..f6b9768
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/int-tests/state-management.xml
@@ -0,0 +1,24 @@
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+      http://www.apache.org/licenses/LICENSE-2.0
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+<stateManagement>
+    <local-provider>
+        <id>local-provider</id>
+        <class>org.apache.nifi.controller.state.providers.local.WriteAheadLocalStateProvider</class>
+        <property name="Directory">./target/int-tests/state/local</property>
+        <property name="Always Sync">false</property>
+        <property name="Partitions">16</property>
+        <property name="Checkpoint Interval">2 mins</property>
+    </local-provider>
+</stateManagement>
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/logback-test.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/logback-test.xml
index f6cb8c5..d8801ce 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/logback-test.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/logback-test.xml
@@ -30,6 +30,7 @@
 
 
     <logger name="org.apache.nifi" level="INFO"/>
+    <logger name="org.apache.nifi.controller.tasks" level="DEBUG" />"
     <logger name="org.apache.nifi.controller.service" level="DEBUG"/>
     <logger name="org.apache.nifi.encrypt" level="DEBUG"/>
     <logger name="org.apache.nifi.controller.service.mock" level="ERROR"/>
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/StandardExtensionDiscoveringManager.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/StandardExtensionDiscoveringManager.java
index 8567f32..adadea1 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/StandardExtensionDiscoveringManager.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/StandardExtensionDiscoveringManager.java
@@ -151,37 +151,7 @@ public class StandardExtensionDiscoveringManager implements ExtensionDiscovering
             final ServiceLoader<?> serviceLoader = ServiceLoader.load(entry.getKey(), bundle.getClassLoader());
             for (final Object o : serviceLoader) {
                 try {
-                    // create a cache of temp ConfigurableComponent instances, the initialize here has to happen before the checks below
-                    if ((isControllerService || isProcessor || isReportingTask) && o instanceof ConfigurableComponent) {
-                        final ConfigurableComponent configurableComponent = (ConfigurableComponent) o;
-                        initializeTempComponent(configurableComponent);
-
-                        final String cacheKey = getClassBundleKey(o.getClass().getCanonicalName(), bundle.getBundleDetails().getCoordinate());
-                        tempComponentLookup.put(cacheKey, (ConfigurableComponent) o);
-                    }
-
-                    // only consider extensions discovered directly in this bundle
-                    boolean registerExtension = bundle.getClassLoader().equals(o.getClass().getClassLoader());
-
-                    if (registerExtension) {
-                        final Class extensionType = o.getClass();
-                        if (isControllerService && !checkControllerServiceEligibility(extensionType)) {
-                            registerExtension = false;
-                            logger.error(String.format(
-                                    "Skipping Controller Service %s because it is bundled with its supporting APIs and requires instance class loading.", extensionType.getName()));
-                        }
-
-                        final boolean canReferenceControllerService = (isControllerService || isProcessor || isReportingTask) && o instanceof ConfigurableComponent;
-                        if (canReferenceControllerService && !checkControllerServiceReferenceEligibility((ConfigurableComponent) o, bundle.getClassLoader())) {
-                            registerExtension = false;
-                            logger.error(String.format(
-                                    "Skipping component %s because it is bundled with its referenced Controller Service APIs and requires instance class loading.", extensionType.getName()));
-                        }
-
-                        if (registerExtension) {
-                            registerServiceClass(o.getClass(), classNameBundleLookup, bundleCoordinateClassesLookup, bundle, entry.getValue());
-                        }
-                    }
+                    loadExtension(o, entry.getKey(), bundle);
                 } catch (Exception e) {
                     logger.warn("Failed to register extension {} due to: {}" , new Object[]{o.getClass().getCanonicalName(), e.getMessage()});
                     if (logger.isDebugEnabled()) {
@@ -194,6 +164,50 @@ public class StandardExtensionDiscoveringManager implements ExtensionDiscovering
         }
     }
 
+    protected void loadExtension(final Object extension, final Class<?> extensionType, final Bundle bundle) {
+        final boolean isControllerService = ControllerService.class.equals(extensionType);
+        final boolean isProcessor = Processor.class.equals(extensionType);
+        final boolean isReportingTask = ReportingTask.class.equals(extensionType);
+
+        // create a cache of temp ConfigurableComponent instances, the initialize here has to happen before the checks below
+        if ((isControllerService || isProcessor || isReportingTask) && extension instanceof ConfigurableComponent) {
+            final ConfigurableComponent configurableComponent = (ConfigurableComponent) extension;
+            initializeTempComponent(configurableComponent);
+
+            final String cacheKey = getClassBundleKey(extension.getClass().getCanonicalName(), bundle.getBundleDetails().getCoordinate());
+            tempComponentLookup.put(cacheKey, configurableComponent);
+        }
+
+        // only consider extensions discovered directly in this bundle
+        boolean registerExtension = bundle.getClassLoader().equals(extension.getClass().getClassLoader());
+
+        if (registerExtension) {
+            final Class<?> extensionClass = extension.getClass();
+            if (isControllerService && !checkControllerServiceEligibility(extensionClass)) {
+                registerExtension = false;
+                logger.error(String.format(
+                    "Skipping Controller Service %s because it is bundled with its supporting APIs and requires instance class loading.", extensionClass.getName()));
+            }
+
+            final boolean canReferenceControllerService = (isControllerService || isProcessor || isReportingTask) && extension instanceof ConfigurableComponent;
+            if (canReferenceControllerService && !checkControllerServiceReferenceEligibility((ConfigurableComponent) extension, bundle.getClassLoader())) {
+                registerExtension = false;
+                logger.error(String.format(
+                    "Skipping component %s because it is bundled with its referenced Controller Service APIs and requires instance class loading.", extensionClass.getName()));
+            }
+
+            if (registerExtension) {
+                registerExtensionClass(extensionType, extension.getClass(), bundle);
+            }
+        }
+    }
+
+    protected void registerExtensionClass(final Class<?> extensionType, final Class<?> implementationClass, final Bundle bundle) {
+        final Set<Class> registeredClasses = definitionMap.get(extensionType);
+        registerServiceClass(implementationClass, classNameBundleLookup, bundleCoordinateClassesLookup, bundle, registeredClasses);
+    }
+
+
     private void initializeTempComponent(final ConfigurableComponent configurableComponent) {
         ConfigurableComponentInitializer initializer = null;
         try {
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-repository-models/src/main/java/org/apache/nifi/controller/repository/claim/StandardContentClaim.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-repository-models/src/main/java/org/apache/nifi/controller/repository/claim/StandardContentClaim.java
index 39a2591..7a3e24f 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-repository-models/src/main/java/org/apache/nifi/controller/repository/claim/StandardContentClaim.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-repository-models/src/main/java/org/apache/nifi/controller/repository/claim/StandardContentClaim.java
@@ -70,6 +70,10 @@ public final class StandardContentClaim implements ContentClaim, Comparable<Cont
             return false;
         }
 
+        if (length != other.getLength()) {
+            return false;
+        }
+
         return resourceClaim.equals(other.getResourceClaim());
     }
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/HttpRemoteSiteListener.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/HttpRemoteSiteListener.java
index deb25a7..4ae8009 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/HttpRemoteSiteListener.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/HttpRemoteSiteListener.java
@@ -25,9 +25,7 @@ import org.apache.nifi.util.NiFiProperties;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.IOException;
 import java.util.Map;
-import java.util.Set;
 import java.util.UUID;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.Executors;
@@ -35,7 +33,6 @@ import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.ScheduledFuture;
 import java.util.concurrent.ThreadFactory;
 import java.util.concurrent.TimeUnit;
-import java.util.stream.Collectors;
 
 import static org.apache.nifi.util.NiFiProperties.DEFAULT_SITE_TO_SITE_HTTP_TRANSACTION_TTL;
 import static org.apache.nifi.util.NiFiProperties.SITE_TO_SITE_HTTP_TRANSACTION_TTL;
@@ -119,15 +116,17 @@ public class HttpRemoteSiteListener implements RemoteSiteListener {
     }
 
     @Override
-    public void start() throws IOException {
+    public void start() {
         transactionMaintenanceTask = taskExecutor.scheduleWithFixedDelay(() -> {
 
             int originalSize = transactions.size();
             logger.trace("Transaction maintenance task started.");
             try {
-                Set<String> transactionIds = transactions.keySet().stream().collect(Collectors.toSet());
-                transactionIds.stream().filter(tid -> !isTransactionActive(tid))
-                        .forEach(tid -> cancelTransaction(tid));
+                for (final String transactionId : transactions.keySet()) {
+                    if (!isTransactionActive(transactionId)) {
+                        cancelTransaction(transactionId);
+                    }
+                }
             } catch (Exception e) {
                 // Swallow exception so that this thread can keep working.
                 logger.error("An exception occurred while maintaining transactions", e);
@@ -168,6 +167,12 @@ public class HttpRemoteSiteListener implements RemoteSiteListener {
         }
     }
 
+    @Override
+    public void destroy() {
+        stop();
+        instance = null;
+    }
+
     public String createTransaction() {
         final String transactionId = UUID.randomUUID().toString();
         transactions.put(transactionId, new TransactionWrapper(null, null));
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/RemoteSiteListener.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/RemoteSiteListener.java
index 1183fc5..888f9ea 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/RemoteSiteListener.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/RemoteSiteListener.java
@@ -16,10 +16,10 @@
  */
 package org.apache.nifi.remote;
 
-import java.io.IOException;
-
 import org.apache.nifi.groups.ProcessGroup;
 
+import java.io.IOException;
+
 public interface RemoteSiteListener {
 
     void setRootGroup(ProcessGroup rootGroup);
@@ -27,4 +27,6 @@ public interface RemoteSiteListener {
     void start() throws IOException;
 
     void stop();
+
+    void destroy();
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/SocketRemoteSiteListener.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/SocketRemoteSiteListener.java
index 07c5920..f864ab7 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/SocketRemoteSiteListener.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/SocketRemoteSiteListener.java
@@ -57,8 +57,6 @@ import java.util.concurrent.atomic.AtomicReference;
 
 public class SocketRemoteSiteListener implements RemoteSiteListener {
 
-    public static final String DEFAULT_FLOWFILE_PATH = "./";
-
     private final int socketPort;
     private final SSLContext sslContext;
     private final NodeInformant nodeInformant;
@@ -375,6 +373,10 @@ public class SocketRemoteSiteListener implements RemoteSiteListener {
         stopped.set(true);
     }
 
+    @Override
+    public void destroy() {
+    }
+
     private void verifyMagicBytes(final InputStream in, final String peerDescription) throws IOException, HandshakeException {
         final byte[] receivedMagicBytes = new byte[CommunicationsSession.MAGIC_BYTES.length];
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardProcessGroupDAO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardProcessGroupDAO.java
index c2a180a..20669f6 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardProcessGroupDAO.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardProcessGroupDAO.java
@@ -361,7 +361,7 @@ public class StandardProcessGroupDAO extends ComponentDAO implements ProcessGrou
 
         final ProcessGroup group = locateProcessGroup(flowController, groupId);
         group.updateFlow(proposedSnapshot, componentIdSeed, verifyNotModified, updateSettings, updateDescendantVersionedFlows);
-        group.findAllRemoteProcessGroups().stream().forEach(RemoteProcessGroup::initialize);
+        group.findAllRemoteProcessGroups().forEach(RemoteProcessGroup::initialize);
 
         final StandardVersionControlInformation svci = StandardVersionControlInformation.Builder.fromDto(versionControlInformation)
             .flowSnapshot(proposedSnapshot.getFlowContents())
diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/lucene/EventIndexTask.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/lucene/EventIndexTask.java
index f8bbd3b..df752b8 100644
--- a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/lucene/EventIndexTask.java
+++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/lucene/EventIndexTask.java
@@ -17,16 +17,6 @@
 
 package org.apache.nifi.provenance.index.lucene;
 
-import java.io.File;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-import java.util.Optional;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.TimeUnit;
-import java.util.stream.Collectors;
-
 import org.apache.lucene.document.Document;
 import org.apache.lucene.search.NumericRangeQuery;
 import org.apache.nifi.events.EventReporter;
@@ -38,6 +28,18 @@ import org.apache.nifi.reporting.Severity;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
 public class EventIndexTask implements Runnable {
     private static final Logger logger = LoggerFactory.getLogger(EventIndexTask.class);
     private static final String EVENT_CATEGORY = "Provenance Repository";
@@ -52,6 +54,8 @@ public class EventIndexTask implements Runnable {
     private final EventReporter eventReporter;
     private final int commitThreshold;
 
+    private volatile CompletableFuture<Void> shutdownComplete;
+
     public EventIndexTask(final BlockingQueue<StoredDocument> documentQueue, final RepositoryConfiguration repoConfig, final IndexManager indexManager,
         final IndexDirectoryManager directoryManager, final int maxEventsPerCommit, final EventReporter eventReporter) {
         this.documentQueue = documentQueue;
@@ -61,8 +65,13 @@ public class EventIndexTask implements Runnable {
         this.eventReporter = eventReporter;
     }
 
-    public void shutdown() {
+    public synchronized Future<Void> shutdown() {
+        if (shutdownComplete == null) {
+            shutdownComplete = new CompletableFuture<>();
+        }
+
         this.shutdown = true;
+        return shutdownComplete;
     }
 
     private void fetchDocuments(final List<StoredDocument> destination) throws InterruptedException {
@@ -108,6 +117,11 @@ public class EventIndexTask implements Runnable {
                 eventReporter.reportEvent(Severity.ERROR, EVENT_CATEGORY, "Failed to index Provenance Events. See logs for more information.");
             }
         }
+
+        final CompletableFuture<Void> future = this.shutdownComplete;
+        if (future != null) {
+            future.complete(null);
+        }
     }
 
 
@@ -119,7 +133,7 @@ public class EventIndexTask implements Runnable {
             return;
         }
 
-        final Map<File, List<IndexableDocument>> docsByIndexDir = toIndex.stream().collect(Collectors.groupingBy(doc -> doc.getIndexDirectory()));
+        final Map<File, List<IndexableDocument>> docsByIndexDir = toIndex.stream().collect(Collectors.groupingBy(IndexableDocument::getIndexDirectory));
         for (final Map.Entry<File, List<IndexableDocument>> entry : docsByIndexDir.entrySet()) {
             final File indexDirectory = entry.getKey();
             final List<IndexableDocument> documentsForIndex = entry.getValue();
@@ -145,7 +159,7 @@ public class EventIndexTask implements Runnable {
                 indexWriter.getIndexWriter().deleteDocuments(query);
 
                 final List<Document> documents = documentsForIndex.stream()
-                    .map(doc -> doc.getDocument())
+                    .map(IndexableDocument::getDocument)
                     .collect(Collectors.toList());
 
                 indexWriter.index(documents, commitThreshold);
diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/lucene/LuceneEventIndex.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/lucene/LuceneEventIndex.java
index d1a9714..87b57c5 100644
--- a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/lucene/LuceneEventIndex.java
+++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/lucene/LuceneEventIndex.java
@@ -17,25 +17,6 @@
 
 package org.apache.nifi.provenance.index.lucene;
 
-import java.io.File;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Date;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Optional;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicLong;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.Term;
@@ -75,6 +56,27 @@ import org.apache.nifi.util.timebuffer.TimestampedLong;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
 
 public class LuceneEventIndex implements EventIndex {
     private static final Logger logger = LoggerFactory.getLogger(LuceneEventIndex.class);
@@ -188,9 +190,21 @@ public class LuceneEventIndex implements EventIndex {
             maintenanceExecutor.shutdown();
         }
 
+        final List<Future<?>> futures = new ArrayList<>();
         for (final EventIndexTask task : indexTasks) {
-            task.shutdown();
+            futures.add(task.shutdown());
         }
+
+        // Wait for all tasks to complete before returning
+        for (final Future<?> future : futures) {
+            try {
+                future.get();
+            } catch (final Exception e) {
+                logger.error("Failed to shutdown Index Task", e);
+            }
+        }
+
+        indexManager.close();
     }
 
     long getMaxEventId(final String partitionName) {
diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/SimpleIndexManager.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/SimpleIndexManager.java
index d59a81d..7dd626b 100644
--- a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/SimpleIndexManager.java
+++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/SimpleIndexManager.java
@@ -17,17 +17,6 @@
 
 package org.apache.nifi.provenance.lucene;
 
-import java.io.Closeable;
-import java.io.File;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.TimeUnit;
-
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.standard.StandardAnalyzer;
 import org.apache.lucene.index.ConcurrentMergeScheduler;
@@ -44,6 +33,19 @@ import org.apache.nifi.provenance.util.NamedThreadFactory;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+
 public class SimpleIndexManager implements IndexManager {
     private static final Logger logger = LoggerFactory.getLogger(SimpleIndexManager.class);
 
@@ -69,6 +71,20 @@ public class SimpleIndexManager implements IndexManager {
             Thread.currentThread().interrupt();
             searchExecutor.shutdownNow();
         }
+
+        synchronized (writerCounts) {
+            final Set<File> closed = new HashSet<>();
+
+            for (final Map.Entry<File, IndexWriterCount> entry : writerCounts.entrySet()) {
+                final IndexWriterCount count = entry.getValue();
+                if (count.getCount() < 1) {
+                    count.close();
+                    closed.add(entry.getKey());
+                }
+            }
+
+            closed.forEach(writerCounts::remove);
+        }
     }
 
     @Override