You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by jg...@apache.org on 2021/10/04 19:32:48 UTC

[nifi] branch main updated: NIFI-9239: Updated Consume/Publish Kafka processors to support Exactly Once Semantics. Added ExecuteStateless processor/bundle - A few minor updates to Stateless in order to surface some concepts from the stateless engine up to the caller, such as bulletins, counters, etc.

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

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


The following commit(s) were added to refs/heads/main by this push:
     new 607480c  NIFI-9239: Updated Consume/Publish Kafka processors to support Exactly Once Semantics. Added ExecuteStateless processor/bundle - A few minor updates to Stateless in order to surface some concepts from the stateless engine up to the caller, such as bulletins, counters, etc.
607480c is described below

commit 607480c1a0f8153f7d1a6470a7b797e3eaed5eb4
Author: Mark Payne <ma...@hotmail.com>
AuthorDate: Thu Sep 16 12:42:55 2021 -0400

    NIFI-9239: Updated Consume/Publish Kafka processors to support Exactly Once Semantics. Added ExecuteStateless processor/bundle
    - A few minor updates to Stateless in order to surface some concepts from the stateless engine up to the caller, such as bulletins, counters, etc.
    
    Signed-off-by: Joe Gresock <jg...@gmail.com>
    
    This closes #5412.
---
 .../main/java/org/apache/nifi/minifi/MiNiFi.java   |    2 +-
 .../apache/nifi/reporting/BulletinRepository.java  |    5 +
 nifi-assembly/pom.xml                              |    6 +
 .../java/org/apache/nifi/util/CapturingLogger.java |   29 +-
 .../apache/nifi/util/MockBulletinRepository.java   |    8 +-
 .../nifi/util/TestStandardProcessorTestRunner.java |   48 +-
 .../kinesis/stream/TestConsumeKinesisStream.java   |   46 -
 .../record/TestAbstractKinesisRecordProcessor.java |   60 +-
 .../record/TestKinesisRecordProcessorRaw.java      |    9 +-
 .../record/TestKinesisRecordProcessorRecord.java   |   14 +-
 .../org/apache/nifi/connectable/LocalPort.java     |   62 +-
 .../nifi/controller/StandardProcessorNode.java     |    7 +-
 .../repository/StandardProcessSession.java         |   17 +-
 .../nifi/events/VolatileBulletinRepository.java    |   10 +
 .../nifi/logging/ControllerServiceLogObserver.java |    0
 .../apache/nifi/logging/ProcessorLogObserver.java  |    0
 .../nifi/logging/ReportingTaskLogObserver.java     |    0
 .../apache/nifi/processor/SimpleProcessLogger.java |   14 +-
 .../java/org/apache/nifi/util/ReflectionUtils.java |    5 +-
 .../nifi/controller/scheduling/LifecycleState.java |   10 +-
 .../apache/nifi/logging/LogRepositoryFactory.java  |   12 +-
 .../nifi/controller/tasks/ConnectableTask.java     |   13 +-
 .../nifi/nar/ExtensionDiscoveringManager.java      |   14 +-
 .../nar/StandardExtensionDiscoveringManager.java   |    6 +-
 .../java/org/apache/nifi/nar/NarClassLoaders.java  |   44 +-
 .../main/java/org/apache/nifi/nar/NarUnpacker.java |    4 +-
 .../src/main/resources/conf/logback.xml            |    1 +
 .../src/main/java/org/apache/nifi/NiFi.java        |    3 +-
 .../processors/hive/TestPutHive3Streaming.java     |   29 +-
 .../kafka/pubsub/ConsumeKafkaRecord_2_6.java       |   29 +-
 .../processors/kafka/pubsub/ConsumeKafka_2_6.java  |   51 +-
 .../processors/kafka/pubsub/ConsumerLease.java     |  118 ++-
 .../nifi/processors/kafka/pubsub/ConsumerPool.java |   31 +-
 .../kafka/pubsub/KafkaProcessorUtils.java          |   21 +-
 .../kafka/pubsub/PublishKafkaRecord_2_6.java       |   14 +-
 .../processors/kafka/pubsub/PublishKafkaUtil.java  |  124 +++
 .../processors/kafka/pubsub/PublishKafka_2_6.java  |   30 +-
 .../processors/kafka/pubsub/PublisherLease.java    |   19 +-
 .../processors/kafka/pubsub/PublisherPool.java     |   15 +
 .../processors/kafka/pubsub/ConsumerPoolTest.java  |   16 +-
 .../nifi/processors/standard/TestPutEmail.java     |   28 +-
 .../nifi-stateless-processor-nar/pom.xml           |   43 +
 .../nifi-stateless-processor-tests/pom.xml         |  170 ++++
 .../src/test/assembly/dependencies.xml             |   62 ++
 .../processors/stateless/TestExecuteStateless.java |  207 ++++
 .../src/test/resources/log-message.json            |    1 +
 .../src/test/resources/passthrough-flow.json       |   90 ++
 .../src/test/resources/route-one-to-failure.json   |    1 +
 .../src/test/resources/route-to-desired-port.json  |    1 +
 .../resources/route-to-failure-inner-group.json    |    1 +
 .../src/test/resources/sleep.json                  |    1 +
 .../src/test/resources/split-text.json             |    1 +
 .../src/test/resources/throw-exception.json        |    1 +
 .../nifi-stateless-processor/pom.xml               |   71 ++
 .../processors/stateless/ExecuteStateless.java     | 1041 ++++++++++++++++++++
 .../retrieval/CachingDataflowProvider.java         |  100 ++
 .../stateless/retrieval/DataflowProvider.java      |   11 +-
 .../retrieval/FileSystemDataflowProvider.java      |   40 +
 .../retrieval/RegistryDataflowProvider.java        |  115 +++
 .../services/org.apache.nifi.processor.Processor   |   16 +
 .../additionalDetails.html                         |  358 +++++++
 .../retrieval/TestCachingDataflowProvider.java     |  130 +++
 .../nifi-stateless-processor-bundle/pom.xml        |   34 +
 nifi-nar-bundles/pom.xml                           |    1 +
 .../org/apache/nifi/registry/link/LinkAdapter.java |    4 +
 .../engine/StatelessEngineConfiguration.java       |    4 +
 ...dException.java => DataflowTriggerContext.java} |   20 +-
 .../flow/FailurePortEncounteredException.java      |    8 +-
 .../nifi/stateless/flow/StatelessDataflow.java     |   24 +-
 .../apache/nifi/stateless/flow/TriggerResult.java  |    7 +
 .../stateless/bootstrap/StatelessBootstrap.java    |   22 +-
 .../stateless/bootstrap/ExtensionDiscovery.java    |    7 +-
 .../nifi/stateless/engine/ExecutionProgress.java   |    6 +
 .../engine/StandardExecutionProgress.java          |   46 +-
 .../stateless/engine/StandardStatelessEngine.java  |   10 +-
 .../stateless/engine/StatelessFlowManager.java     |   32 +-
 .../nifi/stateless/flow/CanceledTriggerResult.java |    6 +
 .../stateless/flow/ExceptionalTriggerResult.java   |    6 +
 .../flow/StandardStatelessDataflowFactory.java     |   17 +-
 .../nifi/stateless/flow/StandardStatelessFlow.java |   86 +-
 .../flow/StandardStatelessFlowCurrent.java         |   21 +-
 .../repository/RepositoryContextFactory.java       |    6 +
 .../StatelessFileSystemContentRepository.java      |    6 +
 .../StatelessRepositoryContextFactory.java         |   10 +
 .../stateless/session/StatelessProcessSession.java |   57 +-
 .../TestStatelessFileSystemContentRepository.java  |   49 +
 .../apache/nifi/stateless/StatelessSystemIT.java   |    9 +-
 .../nifi/stateless/basics/BatchOutputIT.java       |  120 +++
 .../stateless/basics/PartitioningContentIT.java    |   88 ++
 .../stateless/basics/RollbackOnExceptionIT.java    |    6 +-
 .../processors/tests/system/PartitionText.java     |  119 +++
 .../tests/system/RoundRobinFlowFiles.java          |   87 ++
 .../processors/tests/system/TransferBatch.java     |  102 ++
 .../services/org.apache.nifi.processor.Processor   |    3 +
 94 files changed, 4016 insertions(+), 442 deletions(-)

diff --git a/minifi/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-runtime/src/main/java/org/apache/nifi/minifi/MiNiFi.java b/minifi/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-runtime/src/main/java/org/apache/nifi/minifi/MiNiFi.java
index 27eab07..c02513d 100644
--- a/minifi/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-runtime/src/main/java/org/apache/nifi/minifi/MiNiFi.java
+++ b/minifi/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-runtime/src/main/java/org/apache/nifi/minifi/MiNiFi.java
@@ -130,7 +130,7 @@ public class MiNiFi {
         // load the extensions classloaders
         NarClassLoaders narClassLoaders = NarClassLoadersHolder.getInstance();
         narClassLoaders.init(rootClassLoader,
-                properties.getFrameworkWorkingDirectory(), properties.getExtensionsWorkingDirectory(), FRAMEWORK_NAR_ID);
+                properties.getFrameworkWorkingDirectory(), properties.getExtensionsWorkingDirectory(), FRAMEWORK_NAR_ID, true);
 
         /// load the framework classloader
         final ClassLoader frameworkClassLoader = narClassLoaders.getFrameworkBundle().getClassLoader();
diff --git a/nifi-api/src/main/java/org/apache/nifi/reporting/BulletinRepository.java b/nifi-api/src/main/java/org/apache/nifi/reporting/BulletinRepository.java
index be0e3f4..2d731fb 100644
--- a/nifi-api/src/main/java/org/apache/nifi/reporting/BulletinRepository.java
+++ b/nifi-api/src/main/java/org/apache/nifi/reporting/BulletinRepository.java
@@ -90,4 +90,9 @@ public interface BulletinRepository {
      * @return all bulletins for the controller
      */
     List<Bulletin> findBulletinsForController(int max);
+
+    /**
+     * @return the max ID of any bulletin that has been added, or -1 if no bulletins have been added
+     */
+    long getMaxBulletinId();
 }
diff --git a/nifi-assembly/pom.xml b/nifi-assembly/pom.xml
index a1cf0bd..e8402da 100644
--- a/nifi-assembly/pom.xml
+++ b/nifi-assembly/pom.xml
@@ -772,6 +772,12 @@ language governing permissions and limitations under the License. -->
             <version>1.15.0-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-stateless-processor-nar</artifactId>
+            <version>1.15.0-SNAPSHOT</version>
+            <type>nar</type>
+        </dependency>
         <!-- dependencies for jaxb/activation/annotation for running NiFi on Java 11 -->
         <!-- TODO: remove these once minimum Java version is 11 -->
         <dependency>
diff --git a/nifi-mock/src/main/java/org/apache/nifi/util/CapturingLogger.java b/nifi-mock/src/main/java/org/apache/nifi/util/CapturingLogger.java
index c11860a..10af935 100644
--- a/nifi-mock/src/main/java/org/apache/nifi/util/CapturingLogger.java
+++ b/nifi-mock/src/main/java/org/apache/nifi/util/CapturingLogger.java
@@ -1,12 +1,13 @@
 package org.apache.nifi.util;
 
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
 import org.slf4j.Logger;
 import org.slf4j.Marker;
 import org.slf4j.helpers.MessageFormatter;
 
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -361,24 +362,28 @@ public class CapturingLogger implements Logger {
 
     @Override
     public void error(String format, Object arg) {
-        errorMessages.add(new LogMessage(null, format, null, arg));
+        final String message = MessageFormatter.arrayFormat(format, new Object[] {arg}).getMessage();
+        errorMessages.add(new LogMessage(null, message, null, arg));
         logger.error(format, arg);
     }
 
     @Override
     public void error(String format, Object arg1, Object arg2) {
-        errorMessages.add(new LogMessage(null, format, null, arg1, arg2));
+        final String message = MessageFormatter.arrayFormat(format, new Object[] {arg1, arg2}).getMessage();
+        errorMessages.add(new LogMessage(null, message, null, arg1, arg2));
         logger.error(format, arg1, arg2);
     }
 
     public void error(String format, Object arg1, Throwable t) {
-        errorMessages.add(new LogMessage(null, format, t, arg1));
+        final String message = MessageFormatter.arrayFormat(format, new Object[] {arg1}).getMessage();
+        errorMessages.add(new LogMessage(null, message, t, arg1));
         logger.error(format, arg1, t);
     }
 
     @Override
     public void error(String format, Object... arguments) {
-        errorMessages.add(new LogMessage(null, format, null, arguments));
+        final String message = MessageFormatter.arrayFormat(format, arguments).getMessage();
+        errorMessages.add(new LogMessage(null, message, null, arguments));
         logger.error(format, arguments);
     }
 
@@ -402,20 +407,22 @@ public class CapturingLogger implements Logger {
 
     @Override
     public void error(Marker marker, String format, Object arg) {
-        errorMessages.add(new LogMessage(marker, format, null, arg));
+        final String message = MessageFormatter.arrayFormat(format, new Object[] {arg}).getMessage();
+        errorMessages.add(new LogMessage(marker, message, null, arg));
         logger.error(marker, format, arg);
-
     }
 
     @Override
     public void error(Marker marker, String format, Object arg1, Object arg2) {
-        errorMessages.add(new LogMessage(marker, format, null, arg1, arg2));
+        final String message = MessageFormatter.arrayFormat(format, new Object[] {arg1, arg2}).getMessage();
+        errorMessages.add(new LogMessage(marker, message, null, arg1, arg2));
         logger.error(marker, format, arg1, arg2);
     }
 
     @Override
     public void error(Marker marker, String format, Object... argArray) {
-        errorMessages.add(new LogMessage(marker, format, null, argArray));
+        final String message = MessageFormatter.arrayFormat(format, argArray).getMessage();
+        errorMessages.add(new LogMessage(marker, message, null, argArray));
         logger.error(marker, format, argArray);
     }
 
diff --git a/nifi-mock/src/main/java/org/apache/nifi/util/MockBulletinRepository.java b/nifi-mock/src/main/java/org/apache/nifi/util/MockBulletinRepository.java
index 89a0cf9..0ffd0f4 100644
--- a/nifi-mock/src/main/java/org/apache/nifi/util/MockBulletinRepository.java
+++ b/nifi-mock/src/main/java/org/apache/nifi/util/MockBulletinRepository.java
@@ -16,11 +16,12 @@
  */
 package org.apache.nifi.util;
 
-import java.util.List;
 import org.apache.nifi.reporting.Bulletin;
 import org.apache.nifi.reporting.BulletinQuery;
 import org.apache.nifi.reporting.BulletinRepository;
 
+import java.util.List;
+
 public class MockBulletinRepository implements BulletinRepository {
 
     @Override
@@ -76,4 +77,9 @@ public class MockBulletinRepository implements BulletinRepository {
         // TODO: Implement
         return null;
     }
+
+    @Override
+    public long getMaxBulletinId() {
+        return -1L;
+    }
 }
diff --git a/nifi-mock/src/test/java/org/apache/nifi/util/TestStandardProcessorTestRunner.java b/nifi-mock/src/test/java/org/apache/nifi/util/TestStandardProcessorTestRunner.java
index b7025f8..a55462d 100644
--- a/nifi-mock/src/test/java/org/apache/nifi/util/TestStandardProcessorTestRunner.java
+++ b/nifi-mock/src/test/java/org/apache/nifi/util/TestStandardProcessorTestRunner.java
@@ -16,21 +16,6 @@
  */
 package org.apache.nifi.util;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertThrows;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.function.Predicate;
 import org.apache.nifi.annotation.lifecycle.OnStopped;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationResult;
@@ -46,6 +31,23 @@ import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.reporting.InitializationException;
 import org.junit.jupiter.api.Test;
 
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Predicate;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertThrows;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
 public class TestStandardProcessorTestRunner {
 
     @Test
@@ -382,20 +384,16 @@ public class TestStandardProcessorTestRunner {
 
     @Test
     public void testErrorLogMessageArguments() {
-
-        String compName = "name of component";
+        final String compName = "name of component";
         final MockComponentLog logger = new MockComponentLog("first id",compName);
 
-        Throwable t = new ArithmeticException();
+        final Throwable t = new RuntimeException("Intentional Exception for testing purposes");
         logger.error("expected test error",t);
 
-        String expected_throwable = "java.lang.ArithmeticException";
-
-        List<LogMessage>  log = logger.getErrorMessages();
-        LogMessage msg = log.get(0);
-        // checking if the error messages are recorded in the correct throwable argument.
-        assertEquals(expected_throwable,msg.getThrowable().toString());
-        assertEquals("{} expected test error",msg.getMsg());
+        final List<LogMessage>  log = logger.getErrorMessages();
+        final LogMessage msg = log.get(0);
 
+        assertTrue(msg.getMsg().contains("expected test error"));
+        assertNotNull(msg.getThrowable());
     }
 }
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/kinesis/stream/TestConsumeKinesisStream.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/kinesis/stream/TestConsumeKinesisStream.java
index 95a2a0e..1088580 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/kinesis/stream/TestConsumeKinesisStream.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/kinesis/stream/TestConsumeKinesisStream.java
@@ -46,7 +46,6 @@ import static org.hamcrest.CoreMatchers.anyOf;
 import static org.hamcrest.CoreMatchers.containsString;
 import static org.hamcrest.CoreMatchers.equalTo;
 import static org.hamcrest.CoreMatchers.instanceOf;
-import static org.hamcrest.CoreMatchers.is;
 import static org.hamcrest.CoreMatchers.nullValue;
 import static org.hamcrest.CoreMatchers.startsWith;
 import static org.hamcrest.MatcherAssert.assertThat;
@@ -84,12 +83,6 @@ public class TestConsumeKinesisStream {
         runner.assertValid();
 
         ((ConsumeKinesisStream) runner.getProcessor()).onScheduled(runner.getProcessContext());
-        assertThat(runner.getLogger().getInfoMessages().stream()
-                .anyMatch(logMessage -> logMessage.getMsg().endsWith("Creating client using aws credentials provider")), is(true));
-
-        // "raw" credentials aren't used
-        assertThat(runner.getLogger().getInfoMessages().stream()
-                .anyMatch(logMessage -> logMessage.getMsg().endsWith("Creating client using aws credentials")), is(false));
     }
 
     @Test
@@ -397,52 +390,13 @@ public class TestConsumeKinesisStream {
         assertKinesisClientLibConfiguration(processor.kinesisClientLibConfiguration, withCredentials, hostname);
         assertThat(processor.workerBuilder.build().getApplicationName(), equalTo("test-application"));
 
-        // confirm the Kinesis Worker initialisation was attempted
-        assertThat(mockConsumeKinesisStreamRunner.getLogger().getInfoMessages().stream()
-                .anyMatch(logMessage -> logMessage.getMsg().contains(String.format(
-                        "Kinesis Worker prepared for application %s to process stream %s as worker ID %s:",
-                        "test-application", "test-stream", hostname
-                ))), is(true));
-
-        // confirm the processor worked through the onTrigger method (and no execution of stopConsuming method)
-        assertThat(mockConsumeKinesisStreamRunner.getLogger().getInfoMessages().stream()
-                .anyMatch(logMessage -> logMessage.getMsg().contains(String.format("Starting Kinesis Worker %s", hostname))), is(true));
-        assertThat(mockConsumeKinesisStreamRunner.getLogger().getInfoMessages().stream()
-                .noneMatch(logMessage -> logMessage.getMsg().endsWith("Requesting Kinesis Worker shutdown")), is(true));
-        assertThat(mockConsumeKinesisStreamRunner.getLogger().getInfoMessages().stream()
-                .noneMatch(logMessage -> logMessage.getMsg().endsWith("Kinesis Worker shutdown")), is(true));
-        assertThat(mockConsumeKinesisStreamRunner.getLogger().getWarnMessages().isEmpty(), is(true));
-        assertThat(mockConsumeKinesisStreamRunner.getLogger().getErrorMessages().isEmpty(), is(true));
-
         if (!waitForFailure) {
             // re-trigger the processor to ensure the Worker isn't re-initialised when already running
             mockConsumeKinesisStreamRunner.run(1, false, false);
             assertTrue(((MockProcessContext) mockConsumeKinesisStreamRunner.getProcessContext()).isYieldCalled());
 
-            // "Starting" log count remains at 1 from the initial startup above (the Logger doesn't get reset between processor calls)
-            assertThat(mockConsumeKinesisStreamRunner.getLogger().getInfoMessages().stream()
-                    .filter(logMessage -> logMessage.getMsg().contains(String.format("Starting Kinesis Worker %s", hostname))).count(), is(1L));
-            assertThat(mockConsumeKinesisStreamRunner.getLogger().getWarnMessages().isEmpty(), is(true));
-            assertThat(mockConsumeKinesisStreamRunner.getLogger().getErrorMessages().isEmpty(), is(true));
-
             // stop the processor
             mockConsumeKinesisStreamRunner.stop();
-
-            // confirm the processor worked through the stopConsuming method
-            assertThat(mockConsumeKinesisStreamRunner.getLogger().getInfoMessages().stream()
-                    .anyMatch(logMessage -> logMessage.getMsg().endsWith("Requesting Kinesis Worker shutdown")), is(true));
-            assertThat(mockConsumeKinesisStreamRunner.getLogger().getInfoMessages().stream()
-                    .anyMatch(logMessage -> logMessage.getMsg().endsWith("Kinesis Worker shutdown")), is(true));
-
-            // LeaseCoordinator doesn't startup properly (can't create DynamoDB table during unit test) and therefore has a problem during shutdown
-            assertThat(mockConsumeKinesisStreamRunner.getLogger().getWarnMessages().size(), is(2));
-            assertThat(mockConsumeKinesisStreamRunner.getLogger().getWarnMessages().stream()
-                    .anyMatch(logMessage -> logMessage.getMsg().endsWith(
-                            "Problem while shutting down Kinesis Worker: java.lang.NullPointerException: java.util.concurrent.ExecutionException: java.lang.NullPointerException"
-                    )), is(true));
-            assertThat(mockConsumeKinesisStreamRunner.getLogger().getWarnMessages().stream()
-                    .anyMatch(logMessage -> logMessage.getMsg().endsWith("One or more problems while shutting down Kinesis Worker, see logs for details")), is(true));
-            assertThat(mockConsumeKinesisStreamRunner.getLogger().getErrorMessages().isEmpty(), is(true));
         } else {
             for (int runs = 0; runs < 10; runs++) {
                 try {
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/kinesis/stream/record/TestAbstractKinesisRecordProcessor.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/kinesis/stream/record/TestAbstractKinesisRecordProcessor.java
index 44611ef..91e2b54 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/kinesis/stream/record/TestAbstractKinesisRecordProcessor.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/kinesis/stream/record/TestAbstractKinesisRecordProcessor.java
@@ -48,6 +48,8 @@ import java.util.concurrent.atomic.AtomicLong;
 import static org.hamcrest.CoreMatchers.equalTo;
 import static org.hamcrest.CoreMatchers.is;
 import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
 import static org.mockito.Mockito.doThrow;
 import static org.mockito.Mockito.reset;
 import static org.mockito.Mockito.times;
@@ -106,10 +108,6 @@ public class TestAbstractKinesisRecordProcessor {
 
         assertThat(fixture.getNextCheckpointTimeInMillis() > System.currentTimeMillis(), is(true));
         assertThat(fixture.getKinesisShardId(), equalTo("shard-id"));
-
-        // DEBUG messages don't have their fields replaced in the MockComponentLog
-        assertThat(runner.getLogger().getDebugMessages().stream().anyMatch(logMessage -> logMessage.getMsg()
-                .endsWith("Initializing record processor for stream: {} / shard: {}; from sequence number: {}")), is(true));
     }
 
     @Test
@@ -125,12 +123,6 @@ public class TestAbstractKinesisRecordProcessor {
 
         assertThat(fixture.getNextCheckpointTimeInMillis() > System.currentTimeMillis(), is(true));
         assertThat(fixture.getKinesisShardId(), equalTo("shard-id"));
-
-        assertThat(runner.getLogger().getWarnMessages().stream().anyMatch(logMessage -> logMessage.getMsg()
-                .contains(String.format(
-                        "Initializing record processor for stream: %s / shard %s; from sequence number: %s; indicates previously uncheckpointed sequence number: %s",
-                        "kinesis-test", "shard-id", esn, prev
-                ))), is(true));
     }
 
     @Test
@@ -143,9 +135,6 @@ public class TestAbstractKinesisRecordProcessor {
         fixture.shutdown(shutdownInput);
 
         verify(checkpointer, times(1)).checkpoint();
-
-        assertThat(runner.getLogger().getDebugMessages().stream().anyMatch(logMessage -> logMessage.getMsg()
-                .endsWith("Checkpointing shard test-shard")), is(true));
     }
 
     @Test
@@ -159,16 +148,6 @@ public class TestAbstractKinesisRecordProcessor {
         fixture.shutdown(shutdownInput);
 
         verify(checkpointer, times(2)).checkpoint();
-
-        assertThat(runner.getLogger().getWarnMessages().stream().anyMatch(logMessage -> logMessage.getMsg()
-                .endsWith(String.format(
-                        "Transient issue when checkpointing - attempt %d of %d: %s: %s",
-                        1, 2, ThrottlingException.class.getName(), "throttled"
-                ))), is(true));
-
-        // ERROR messages don't have their fields replaced in the MockComponentLog
-        assertThat(runner.getLogger().getErrorMessages().stream().anyMatch(logMessage -> logMessage.getMsg()
-                .endsWith("Checkpoint failed after {} attempts.: {}")), is(true));
     }
 
     @Test
@@ -182,9 +161,6 @@ public class TestAbstractKinesisRecordProcessor {
         fixture.shutdown(shutdownInput);
 
         verify(checkpointer, times(1)).checkpoint();
-
-        assertThat(runner.getLogger().getInfoMessages().stream().anyMatch(logMessage -> logMessage.getMsg()
-                .endsWith("Caught shutdown exception, skipping checkpoint.")), is(true));
     }
 
     @Test
@@ -199,8 +175,7 @@ public class TestAbstractKinesisRecordProcessor {
 
         verify(checkpointer, times(1)).checkpoint();
 
-        assertThat(runner.getLogger().getErrorMessages().stream().anyMatch(logMessage -> logMessage.getMsg()
-                .endsWith("Cannot save checkpoint to the DynamoDB table used by the Amazon Kinesis Client Library.")), is(true));
+        assertFalse(runner.getLogger().getErrorMessages().isEmpty());
     }
 
     @Test
@@ -215,18 +190,7 @@ public class TestAbstractKinesisRecordProcessor {
 
         verify(checkpointer, times(1)).checkpoint();
 
-        assertThat(runner.getLogger().getDebugMessages().stream().anyMatch(logMessage -> logMessage.getMsg()
-                .endsWith("Checkpointing shard test-shard")), is(true));
-
-        // DEBUG messages don't have their fields replaced in the MockComponentLog
-        assertThat(runner.getLogger().getDebugMessages().stream().anyMatch(logMessage -> logMessage.getMsg()
-                .endsWith("Shutting down Record Processor for shard: {} with reason: {}")), is(true));
-
-        // no waiting loop when records aren't processing
-        assertThat(runner.getLogger().getDebugMessages().stream().anyMatch(logMessage -> logMessage.getMsg()
-                .endsWith("Record Processor for shard {} still processing records, waiting before shutdown")), is(false));
-
-        assertThat(runner.getLogger().getWarnMessages().size(), is(0));
+        assertTrue(runner.getLogger().getWarnMessages().isEmpty());
     }
 
     @Test
@@ -241,20 +205,6 @@ public class TestAbstractKinesisRecordProcessor {
 
         verify(checkpointer, times(1)).checkpoint();
 
-        assertThat(runner.getLogger().getDebugMessages().stream().anyMatch(logMessage -> logMessage.getMsg()
-                .endsWith("Checkpointing shard test-shard")), is(true));
-
-        // DEBUG messages don't have their fields replaced in the MockComponentLog
-        assertThat(runner.getLogger().getDebugMessages().stream().anyMatch(logMessage -> logMessage.getMsg()
-                .endsWith("Shutting down Record Processor for shard: {} with reason: {}")), is(true));
-
-        // wait loop when records are processing
-        assertThat(runner.getLogger().getDebugMessages().stream().filter(logMessage -> logMessage.getMsg()
-                .endsWith("Record Processor for shard {} still processing records, waiting before shutdown"))
-                .count(), is(2L));
-
-        assertThat(runner.getLogger().getWarnMessages().stream().filter(logMessage -> logMessage.getMsg()
-                .endsWith("Record Processor for shard test-shard still running, but maximum wait time elapsed, checkpoint will be attempted"))
-                .count(), is(1L));
+        assertFalse(runner.getLogger().getWarnMessages().isEmpty());
     }
 }
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/kinesis/stream/record/TestKinesisRecordProcessorRaw.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/kinesis/stream/record/TestKinesisRecordProcessorRaw.java
index c3f3963..4cdddc0 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/kinesis/stream/record/TestKinesisRecordProcessorRaw.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/kinesis/stream/record/TestKinesisRecordProcessorRaw.java
@@ -49,6 +49,7 @@ import java.util.concurrent.atomic.AtomicLong;
 
 import static org.hamcrest.CoreMatchers.is;
 import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNull;
 import static org.mockito.Mockito.reset;
 import static org.mockito.Mockito.times;
@@ -214,12 +215,8 @@ public class TestKinesisRecordProcessorRaw {
         assertNull(verify(kinesisRecord, times(2)).getData());
         verify(checkpointer, times(1)).checkpoint();
 
-        // ERROR messages don't have their fields replaced in the MockComponentLog
-        assertThat(runner.getLogger().getErrorMessages().stream().filter(logMessage -> logMessage.getMsg()
-                .endsWith("Caught Exception while processing Kinesis record {}: {}"))
-                .count(), is(2L));
-        assertThat(runner.getLogger().getErrorMessages().stream().anyMatch(logMessage -> logMessage.getMsg()
-                .endsWith("Couldn't process Kinesis record {}, skipping.")), is(true));
+        assertFalse(runner.getLogger().getErrorMessages().isEmpty());
+        assertFalse(runner.getLogger().getErrorMessages().isEmpty());
 
         session.assertCommitted();
         session.assertNotRolledBack();
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/kinesis/stream/record/TestKinesisRecordProcessorRecord.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/kinesis/stream/record/TestKinesisRecordProcessorRecord.java
index 408df0c..4d5215f 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/kinesis/stream/record/TestKinesisRecordProcessorRecord.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/kinesis/stream/record/TestKinesisRecordProcessorRecord.java
@@ -57,6 +57,8 @@ import java.util.concurrent.atomic.AtomicLong;
 
 import static org.hamcrest.CoreMatchers.is;
 import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNull;
 import static org.mockito.Mockito.reset;
 import static org.mockito.Mockito.times;
@@ -239,12 +241,7 @@ public class TestKinesisRecordProcessorRecord {
         assertNull(verify(kinesisRecord, times(2)).getData());
         verify(checkpointer, times(1)).checkpoint();
 
-        // ERROR messages don't have their fields replaced in the MockComponentLog
-        assertThat(runner.getLogger().getErrorMessages().stream().filter(logMessage -> logMessage.getMsg()
-                .endsWith("Caught Exception while processing Kinesis record {}: {}"))
-                .count(), is(2L));
-        assertThat(runner.getLogger().getErrorMessages().stream().anyMatch(logMessage -> logMessage.getMsg()
-                .endsWith("Couldn't process Kinesis record {}, skipping.")), is(true));
+        assertFalse(runner.getLogger().getErrorMessages().isEmpty());
 
         session.assertCommitted();
         session.assertNotRolledBack();
@@ -300,10 +297,7 @@ public class TestKinesisRecordProcessorRecord {
         assertNull(verify(kinesisRecord, times(2)).getData());
         verify(checkpointer, times(1)).checkpoint();
 
-        // ERROR messages don't have their fields replaced in the MockComponentLog
-        assertThat(runner.getLogger().getErrorMessages().stream().filter(logMessage -> logMessage.getMsg()
-                .endsWith("Failed to parse message from Kinesis Stream using configured Record Reader and Writer due to {}: {}"))
-                .count(), is(1L));
+        assertEquals(1, runner.getLogger().getErrorMessages().size());
 
         session.assertCommitted();
         session.assertNotRolledBack();
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/connectable/LocalPort.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/connectable/LocalPort.java
index 9976b4c..96baa05 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/connectable/LocalPort.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/connectable/LocalPort.java
@@ -34,6 +34,7 @@ import org.slf4j.LoggerFactory;
 
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Comparator;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -230,35 +231,54 @@ public class LocalPort extends AbstractPort {
         final Map<String, Integer> counts = getProcessGroup().getBatchCounts().captureCounts();
         counts.forEach((k, v) -> attributes.put("batch.output." + k, String.valueOf(v)));
 
-        Set<Relationship> available = context.getAvailableRelationships();
-        int iterations = 0;
-        while (!available.isEmpty()) {
-            final List<FlowFile> flowFiles = session.get(1000);
-            if (flowFiles.isEmpty()) {
-                break;
-            }
+        final List<Connection> outgoingConnections = new ArrayList<>(getConnections());
 
-            if (!attributes.isEmpty()) {
-                flowFiles.forEach(ff -> session.putAllAttributes(ff, attributes));
-            }
+        final boolean batchOutput = getProcessGroup().getFlowFileOutboundPolicy() == FlowFileOutboundPolicy.BATCH_OUTPUT && getConnectableType() == ConnectableType.OUTPUT_PORT;
+        if (batchOutput) {
+            // Lock the outgoing connections so that the destination of the connection is unable to pull any data until all
+            // data has finished transferring. Before locking, we must sort by identifier so that if there are two arbitrary connections, we always lock them in the same order.
+            // (I.e., anywhere that we lock connections, we must always first sort by ID).
+            // Otherwise, we could encounter a deadlock, if another thread were to lock the same two connections in a different order.
+            outgoingConnections.sort(Comparator.comparing(Connection::getIdentifier));
+            outgoingConnections.forEach(Connection::lock);
+        }
 
-            session.transfer(flowFiles, Relationship.ANONYMOUS);
-            session.commitAsync();
+        try {
+            Set<Relationship> available = context.getAvailableRelationships();
+            int iterations = 0;
+            while (!available.isEmpty()) {
+                final List<FlowFile> flowFiles = session.get(1000);
+                if (flowFiles.isEmpty()) {
+                    break;
+                }
 
-            logger.debug("{} Transferred {} FlowFiles", this, flowFiles.size());
+                if (!attributes.isEmpty()) {
+                    flowFiles.forEach(ff -> session.putAllAttributes(ff, attributes));
+                }
 
-            // If there are fewer than 1,000 FlowFiles available to transfer, or if we
-            // have hit the configured FlowFile cap, we want to stop. This prevents us from
-            // holding the Timer-Driven Thread for an excessive amount of time.
-            if (flowFiles.size() < 1000 || ++iterations >= maxIterations) {
-                break;
-            }
+                session.transfer(flowFiles, Relationship.ANONYMOUS);
+                session.commitAsync();
 
-            available = context.getAvailableRelationships();
+                logger.debug("{} Transferred {} FlowFiles", this, flowFiles.size());
+
+                // If there are fewer than 1,000 FlowFiles available to transfer, or if we
+                // have hit the configured FlowFile cap, we want to stop. This prevents us from
+                // holding the Timer-Driven Thread for an excessive amount of time.
+                if (flowFiles.size() < 1000 || ++iterations >= maxIterations) {
+                    break;
+                }
+
+                available = context.getAvailableRelationships();
+            }
+        } finally {
+            if (batchOutput) {
+                // Reverse ordering in order to unlock.
+                outgoingConnections.sort(Comparator.comparing(Connection::getIdentifier).reversed());
+                outgoingConnections.forEach(Connection::unlock);
+            }
         }
     }
 
-
     @Override
     public void updateConnection(final Connection connection) throws IllegalStateException {
         writeLock.lock();
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java
index 4248cbf..cd55032 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java
@@ -602,11 +602,8 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable
         final long yieldMillis = getYieldPeriod(TimeUnit.MILLISECONDS);
         yield(yieldMillis, TimeUnit.MILLISECONDS);
 
-        final String yieldDuration = (yieldMillis > 1000) ? (yieldMillis / 1000) + " seconds"
-                : yieldMillis + " milliseconds";
-        LoggerFactory.getLogger(processor.getClass()).debug(
-                "{} has chosen to yield its resources; will not be scheduled to run again for {}", processor,
-                yieldDuration);
+        final String yieldDuration = (yieldMillis > 1000) ? (yieldMillis / 1000) + " seconds" : yieldMillis + " milliseconds";
+        LoggerFactory.getLogger(processor.getClass()).trace("{} has chosen to yield its resources; will not be scheduled to run again for {}", processor, yieldDuration);
     }
 
     @Override
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java
index eba58ab..4e7b286 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java
@@ -80,6 +80,7 @@ import java.util.ArrayList;
 import java.util.BitSet;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.Comparator;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
@@ -1754,8 +1755,15 @@ public class StandardProcessSession implements ProcessSession, ProvenanceEventEn
 
 
     private List<FlowFile> get(final ConnectionPoller poller, final boolean lockAllQueues) {
-        final List<Connection> connections = context.getPollableConnections();
-        if (lockAllQueues) {
+        List<Connection> connections = context.getPollableConnections();
+        final boolean sortConnections = lockAllQueues && connections.size() > 1;
+        if (sortConnections) {
+            // Sort by identifier so that if there are two arbitrary connections, we always lock them in the same order.
+            // Otherwise, we could encounter a deadlock, if another thread were to lock the same two connections in a different order.
+            // So we always lock ordered on connection identifier. And unlock in the opposite order.
+            // Before doing this, we must create a copy of the List because the one provided by context.getPollableConnections() is usually an unmodifiableList
+            connections = new ArrayList<>(connections);
+            connections.sort(Comparator.comparing(Connection::getIdentifier));
             for (final Connection connection : connections) {
                 connection.lock();
             }
@@ -1785,7 +1793,10 @@ public class StandardProcessSession implements ProcessSession, ProvenanceEventEn
 
             return new ArrayList<>();
         } finally {
-            if (lockAllQueues) {
+            if (sortConnections) {
+                // Reverse ordering in order to unlock
+                connections.sort(Comparator.comparing(Connection::getIdentifier).reversed());
+
                 for (final Connection connection : connections) {
                     connection.unlock();
                 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/events/VolatileBulletinRepository.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/events/VolatileBulletinRepository.java
index ad4d690..d532625 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/events/VolatileBulletinRepository.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/events/VolatileBulletinRepository.java
@@ -31,6 +31,7 @@ import java.util.TreeSet;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
 
 public class VolatileBulletinRepository implements BulletinRepository {
 
@@ -42,10 +43,19 @@ public class VolatileBulletinRepository implements BulletinRepository {
 
     private final ConcurrentMap<String, ConcurrentMap<String, RingBuffer<Bulletin>>> bulletinStoreMap = new ConcurrentHashMap<>();
     private volatile BulletinProcessingStrategy processingStrategy = new DefaultBulletinProcessingStrategy();
+    private final AtomicLong maxId = new AtomicLong(-1L);
 
     @Override
     public void addBulletin(final Bulletin bulletin) {
         processingStrategy.update(bulletin);
+
+        final long bulletinId = bulletin.getId();
+        maxId.getAndAccumulate(bulletinId, Math::max);
+    }
+
+    @Override
+    public long getMaxBulletinId() {
+        return maxId.get();
     }
 
     @Override
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/logging/ControllerServiceLogObserver.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/logging/ControllerServiceLogObserver.java
similarity index 100%
rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/logging/ControllerServiceLogObserver.java
rename to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/logging/ControllerServiceLogObserver.java
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/logging/ProcessorLogObserver.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/logging/ProcessorLogObserver.java
similarity index 100%
rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/logging/ProcessorLogObserver.java
rename to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/logging/ProcessorLogObserver.java
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/logging/ReportingTaskLogObserver.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/logging/ReportingTaskLogObserver.java
similarity index 100%
rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/logging/ReportingTaskLogObserver.java
rename to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/logging/ReportingTaskLogObserver.java
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/processor/SimpleProcessLogger.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/processor/SimpleProcessLogger.java
index 1c9a26b..85447d1 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/processor/SimpleProcessLogger.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/processor/SimpleProcessLogger.java
@@ -95,7 +95,7 @@ public class SimpleProcessLogger implements ComponentLog {
             return;
         }
 
-        os = addProcessorAndThrowable(os, t, logger.isDebugEnabled());
+        os = addProcessorAndThrowable(os, t);
         msg = "{} " + msg + ": {}";
         logger.warn(msg, os);
         logRepository.addLogMessage(LogLevel.WARN, msg, os, t);
@@ -155,7 +155,7 @@ public class SimpleProcessLogger implements ComponentLog {
             return;
         }
 
-        os = addProcessorAndThrowable(os, t, true);
+        os = addProcessorAndThrowable(os, t);
         msg = "{} " + msg + ": {}";
 
         logger.trace(msg, os);
@@ -235,7 +235,7 @@ public class SimpleProcessLogger implements ComponentLog {
             return;
         }
 
-        os = addProcessorAndThrowable(os, t, logger.isDebugEnabled());
+        os = addProcessorAndThrowable(os, t);
         msg = "{} " + msg + ": {}";
 
         logger.info(msg, os);
@@ -293,16 +293,16 @@ public class SimpleProcessLogger implements ComponentLog {
             return;
         }
 
-        os = addProcessorAndThrowable(os, t, true);
+        os = addProcessorAndThrowable(os, t);
         msg = "{} " + msg + ": {}";
 
         logger.error(msg, os);
         logRepository.addLogMessage(LogLevel.ERROR, msg, os, t);
     }
 
-    private Object[] addProcessorAndThrowable(final Object[] os, final Throwable t, final boolean includeStackTrace) {
+    private Object[] addProcessorAndThrowable(final Object[] os, final Throwable t) {
         final Object[] modifiedArgs;
-        if (t == null || !includeStackTrace) {
+        if (t == null) {
             modifiedArgs = new Object[os.length + 2];
             modifiedArgs[0] = component.toString();
             System.arraycopy(os, 0, modifiedArgs, 1, os.length);
@@ -350,7 +350,7 @@ public class SimpleProcessLogger implements ComponentLog {
             return;
         }
 
-        os = addProcessorAndThrowable(os, t, true);
+        os = addProcessorAndThrowable(os, t);
         msg = "{} " + msg + ": {}";
 
         logger.debug(msg, os);
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/util/ReflectionUtils.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/util/ReflectionUtils.java
index 420fe08..5c5ef9a 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/util/ReflectionUtils.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/util/ReflectionUtils.java
@@ -146,8 +146,9 @@ public class ReflectionUtils {
                 } catch (IllegalAccessException | IllegalArgumentException | InvocationTargetException e) {
                     isSuccess = false;
                     if (quietly) {
+                        final Throwable cause = (e instanceof InvocationTargetException) ? e.getCause() : e;
                         logErrorMessage("Failed while invoking annotated method '" + method + "' with arguments '"
-                            + Arrays.asList(modifiedArgs) + "'.", logger, e);
+                            + Arrays.asList(modifiedArgs) + "'.", logger, cause);
                     } else {
                         throw e;
                     }
@@ -267,7 +268,7 @@ public class ReflectionUtils {
         return updatedArguments;
     }
 
-    private static void logErrorMessage(String message, ComponentLog processLogger, Exception e) {
+    private static void logErrorMessage(String message, ComponentLog processLogger, Throwable e) {
         if (processLogger != null) {
             if (e != null) {
                 processLogger.error(message, e);
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/scheduling/LifecycleState.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/scheduling/LifecycleState.java
index bcc7321..c2dd4e0 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/scheduling/LifecycleState.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/scheduling/LifecycleState.java
@@ -16,6 +16,10 @@
  */
 package org.apache.nifi.controller.scheduling;
 
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.controller.repository.ActiveProcessSessionFactory;
+import org.apache.nifi.processor.exception.TerminatedTaskException;
+
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashSet;
@@ -24,10 +28,6 @@ import java.util.concurrent.ScheduledFuture;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 
-import org.apache.nifi.annotation.lifecycle.OnStopped;
-import org.apache.nifi.controller.repository.ActiveProcessSessionFactory;
-import org.apache.nifi.processor.exception.TerminatedTaskException;
-
 public class LifecycleState {
 
     private final AtomicInteger activeThreadCount = new AtomicInteger(0);
@@ -70,7 +70,7 @@ public class LifecycleState {
         return scheduled.get();
     }
 
-    void setScheduled(final boolean scheduled) {
+    public void setScheduled(final boolean scheduled) {
         this.scheduled.set(scheduled);
         mustCallOnStoppedMethods.set(true);
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/logging/LogRepositoryFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/logging/LogRepositoryFactory.java
index 3b3a072..6b139f7 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/logging/LogRepositoryFactory.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/logging/LogRepositoryFactory.java
@@ -35,8 +35,12 @@ public class LogRepositoryFactory {
         Class<LogRepository> clazz = null;
         try {
             clazz = (Class<LogRepository>) Class.forName(LOG_REPOSITORY_CLASS_NAME, true, LogRepositoryFactory.class.getClassLoader());
-        } catch (ClassNotFoundException e) {
-            LoggerFactory.getLogger(LogRepositoryFactory.class).error("Unable to find class {}; logging may not work properly", LOG_REPOSITORY_CLASS_NAME);
+        } catch (final ClassNotFoundException e) {
+            try {
+                clazz = (Class<LogRepository>) Class.forName(LOG_REPOSITORY_CLASS_NAME, true, Thread.currentThread().getContextClassLoader());
+            } catch (final ClassNotFoundException e1) {
+                LoggerFactory.getLogger(LogRepositoryFactory.class).error("Unable to find class {}; logging may not work properly", LOG_REPOSITORY_CLASS_NAME);
+            }
         }
         logRepositoryClass = clazz;
     }
@@ -59,6 +63,10 @@ public class LogRepositoryFactory {
         return repository;
     }
 
+    public static void purge() {
+        repositoryMap.clear();
+    }
+
     public static LogRepository removeRepository(final String componentId) {
         return repositoryMap.remove(requireNonNull(componentId));
     }
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 916d9a3..5ade471 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
@@ -260,10 +260,15 @@ public class ConnectableTask {
         } finally {
             try {
                 if (batch) {
-                    rawSession.commitAsync(null, t -> {
-                        final ComponentLog procLog = new SimpleProcessLogger(connectable.getIdentifier(), connectable.getRunnableComponent());
-                        procLog.error("Failed to commit session {} due to {}; rolling back", new Object[] { rawSession, t.toString() }, t);
-                    });
+                    final ComponentLog procLog = new SimpleProcessLogger(connectable.getIdentifier(), connectable.getRunnableComponent());
+
+                    try {
+                        rawSession.commitAsync(null, t -> {
+                            procLog.error("Failed to commit session {} due to {}; rolling back", new Object[]{rawSession, t.toString()}, t);
+                        });
+                    } catch (final TerminatedTaskException tte) {
+                        procLog.debug("Cannot commit Batch Process Session because the Task was forcefully terminated", tte);
+                    }
                 }
 
                 final long processingNanos = System.nanoTime() - startNanos;
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/ExtensionDiscoveringManager.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/ExtensionDiscoveringManager.java
index 00b1258..578b2ce 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/ExtensionDiscoveringManager.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/ExtensionDiscoveringManager.java
@@ -42,6 +42,18 @@ public interface ExtensionDiscoveringManager extends ExtensionManager {
      *
      * @param narBundles the bundles to use for discovering extensions
      */
-    void discoverExtensions(Set<Bundle> narBundles);
+    default void discoverExtensions(Set<Bundle> narBundles) {
+        discoverExtensions(narBundles, true);
+    }
+
+    /**
+     * Discovers extensions in the provided bundles.
+     *
+     * This method is intended to be used to discover additional extensions after the application is running.
+     *
+     * @param narBundles the bundles to use for discovering extensions
+     * @param logDetails whether or not to log the details about what is loaded
+     */
+    void discoverExtensions(Set<Bundle> narBundles, boolean logDetails);
 
 }
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 e2d1826..09031a8 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
@@ -129,7 +129,7 @@ public class StandardExtensionDiscoveringManager implements ExtensionDiscovering
     }
 
     @Override
-    public void discoverExtensions(final Set<Bundle> narBundles) {
+    public void discoverExtensions(final Set<Bundle> narBundles, final boolean logDetails) {
         // get the current context class loader
         ClassLoader currentContextClassLoader = Thread.currentThread().getContextClassLoader();
 
@@ -143,7 +143,9 @@ public class StandardExtensionDiscoveringManager implements ExtensionDiscovering
             final long loadStart = System.currentTimeMillis();
             loadExtensions(bundle);
             final long loadMillis = System.currentTimeMillis() - loadStart;
-            logger.info("Loaded extensions for {} in {} millis", bundle.getBundleDetails(), loadMillis);
+            if (logDetails) {
+                logger.info("Loaded extensions for {} in {} millis", bundle.getBundleDetails(), loadMillis);
+            }
 
             // Create a look-up from coordinate to bundle
             bundleCoordinateBundleLookup.put(bundle.getBundleDetails().getCoordinate(), bundle);
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarClassLoaders.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarClassLoaders.java
index 818153b..0f37ade 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarClassLoaders.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarClassLoaders.java
@@ -93,7 +93,7 @@ public final class NarClassLoaders {
      * directories cannot reinitialize or use a different pair of directories.
      */
     public void init(File frameworkWorkingDir, File extensionsWorkingDir, final String frameworkNarId) throws IOException, ClassNotFoundException {
-        init(ClassLoader.getSystemClassLoader(), frameworkWorkingDir, extensionsWorkingDir, frameworkNarId);
+        init(ClassLoader.getSystemClassLoader(), frameworkWorkingDir, extensionsWorkingDir, frameworkNarId, true);
     }
 
     public void init(File frameworkWorkingDir, File extensionsWorkingDir) throws IOException, ClassNotFoundException {
@@ -102,8 +102,8 @@ public final class NarClassLoaders {
 
     // Default to NiFi's framework NAR ID
     public void init(final ClassLoader rootClassloader,
-                     final File frameworkWorkingDir, final File extensionsWorkingDir) throws IOException, ClassNotFoundException {
-        init(rootClassloader, frameworkWorkingDir, extensionsWorkingDir, NarClassLoaders.FRAMEWORK_NAR_ID);
+                     final File frameworkWorkingDir, final File extensionsWorkingDir, final boolean logDetails) throws IOException, ClassNotFoundException {
+        init(rootClassloader, frameworkWorkingDir, extensionsWorkingDir, NarClassLoaders.FRAMEWORK_NAR_ID, logDetails);
     }
 
     /**
@@ -120,8 +120,8 @@ public final class NarClassLoaders {
      * @throws IllegalStateException already initialized with a given pair of
      * directories cannot reinitialize or use a different pair of directories.
      */
-    public void init(final ClassLoader rootClassloader,
-                     final File frameworkWorkingDir, final File extensionsWorkingDir, final String frameworkNarId) throws IOException, ClassNotFoundException {
+    public void init(final ClassLoader rootClassloader, final File frameworkWorkingDir, final File extensionsWorkingDir,
+                     final String frameworkNarId, final boolean logDetails) throws IOException, ClassNotFoundException {
         if (extensionsWorkingDir == null) {
             throw new NullPointerException("cannot have empty arguments");
         }
@@ -131,7 +131,7 @@ public final class NarClassLoaders {
             synchronized (this) {
                 ic = initContext;
                 if (ic == null) {
-                    initContext = ic = load(rootClassloader, frameworkWorkingDir, extensionsWorkingDir, frameworkNarId);
+                    initContext = ic = load(rootClassloader, frameworkWorkingDir, extensionsWorkingDir, frameworkNarId, logDetails);
                 }
             }
         }
@@ -140,8 +140,8 @@ public final class NarClassLoaders {
     /**
      * Should be called at most once.
      */
-    private InitContext load(final ClassLoader rootClassloader,
-                             final File frameworkWorkingDir, final File extensionsWorkingDir, final String frameworkNarId)
+    private InitContext load(final ClassLoader rootClassloader, final File frameworkWorkingDir, final File extensionsWorkingDir,
+                             final String frameworkNarId, final boolean logDetails)
             throws IOException, ClassNotFoundException {
 
         // find all nar files and create class loaders for them.
@@ -203,7 +203,7 @@ public final class NarClassLoaders {
                 // look for the jetty nar
                 if (JETTY_NAR_ID.equals(narDetail.getCoordinate().getId())) {
                     // create the jetty classloader
-                    jettyClassLoader = createNarClassLoader(narDetail.getWorkingDirectory(), rootClassloader);
+                    jettyClassLoader = createNarClassLoader(narDetail.getWorkingDirectory(), rootClassloader, logDetails);
 
                     // remove the jetty nar since its already loaded
                     narDirectoryBundleLookup.put(narDetail.getWorkingDirectory().getCanonicalPath(), new Bundle(narDetail, jettyClassLoader));
@@ -231,14 +231,14 @@ public final class NarClassLoaders {
                     ClassLoader narClassLoader = null;
                     if (narDependencyCoordinate == null) {
                         final ClassLoader parentClassLoader = jettyClassLoader == null ? ClassLoader.getSystemClassLoader() : jettyClassLoader;
-                        narClassLoader = createNarClassLoader(narDetail.getWorkingDirectory(), parentClassLoader);
+                        narClassLoader = createNarClassLoader(narDetail.getWorkingDirectory(), parentClassLoader, logDetails);
                     } else {
                         final String dependencyCoordinateStr = narDependencyCoordinate.getCoordinate();
 
                         // if the declared dependency has already been loaded
                         if (narCoordinateClassLoaderLookup.containsKey(dependencyCoordinateStr)) {
                             final ClassLoader narDependencyClassLoader = narCoordinateClassLoaderLookup.get(dependencyCoordinateStr);
-                            narClassLoader = createNarClassLoader(narDetail.getWorkingDirectory(), narDependencyClassLoader);
+                            narClassLoader = createNarClassLoader(narDetail.getWorkingDirectory(), narDependencyClassLoader, logDetails);
                         } else {
                             // get all bundles that match the declared dependency id
                             final Set<BundleCoordinate> coordinates = narIdBundleLookup.get(narDependencyCoordinate.getId());
@@ -256,7 +256,7 @@ public final class NarClassLoaders {
                                                 narDetail.getCoordinate().getCoordinate(), dependencyCoordinateStr, coordinate.getCoordinate()));
 
                                         final ClassLoader narDependencyClassLoader = narCoordinateClassLoaderLookup.get(coordinate.getCoordinate());
-                                        narClassLoader = createNarClassLoader(narDetail.getWorkingDirectory(), narDependencyClassLoader);
+                                        narClassLoader = createNarClassLoader(narDetail.getWorkingDirectory(), narDependencyClassLoader, logDetails);
                                     }
                                 }
                             }
@@ -355,7 +355,7 @@ public final class NarClassLoaders {
                 final BundleDetails bundleDetail = additionalBundleDetailsIter.next();
                 try {
                     // If we were able to create the bundle class loader, store it and remove the details
-                    final ClassLoader bundleClassLoader = createBundleClassLoader(bundleDetail, bundleIdToCoordinatesLookup);
+                    final ClassLoader bundleClassLoader = createBundleClassLoader(bundleDetail, bundleIdToCoordinatesLookup, true);
                     if (bundleClassLoader != null) {
                         final Bundle bundle = new Bundle(bundleDetail, bundleClassLoader);
                         loadedBundles.add(bundle);
@@ -383,7 +383,7 @@ public final class NarClassLoaders {
         return new NarLoadResult(loadedBundles, skippedBundles);
     }
 
-    private ClassLoader createBundleClassLoader(final BundleDetails bundleDetail, final Map<String,Set<BundleCoordinate>> bundleIdToCoordinatesLookup)
+    private ClassLoader createBundleClassLoader(final BundleDetails bundleDetail, final Map<String,Set<BundleCoordinate>> bundleIdToCoordinatesLookup, final boolean logDetails)
             throws IOException, ClassNotFoundException {
 
         ClassLoader bundleClassLoader = null;
@@ -398,14 +398,14 @@ public final class NarClassLoaders {
                 // If there is no Jetty bundle, assume to be "headless"
                 parentClassLoader = null;
             }
-            bundleClassLoader = createNarClassLoader(bundleDetail.getWorkingDirectory(), parentClassLoader);
+            bundleClassLoader = createNarClassLoader(bundleDetail.getWorkingDirectory(), parentClassLoader, logDetails);
         } else {
             final Optional<Bundle> dependencyBundle = getBundle(bundleDependencyCoordinate);
 
             // If the declared dependency has already been loaded then use it
             if (dependencyBundle.isPresent()) {
                 final ClassLoader narDependencyClassLoader = dependencyBundle.get().getClassLoader();
-                bundleClassLoader = createNarClassLoader(bundleDetail.getWorkingDirectory(), narDependencyClassLoader);
+                bundleClassLoader = createNarClassLoader(bundleDetail.getWorkingDirectory(), narDependencyClassLoader, logDetails);
             } else {
                 // Otherwise get all bundles that match the declared dependency id
                 final Set<BundleCoordinate> coordinates = bundleIdToCoordinatesLookup.get(bundleDependencyCoordinate.getId());
@@ -425,7 +425,7 @@ public final class NarClassLoaders {
                                     bundleDetail.getCoordinate().getCoordinate(), dependencyCoordinateStr, coordinate.getCoordinate()));
 
                             final ClassLoader narDependencyClassLoader = matchingDependencyIdBundle.get().getClassLoader();
-                            bundleClassLoader = createNarClassLoader(bundleDetail.getWorkingDirectory(), narDependencyClassLoader);
+                            bundleClassLoader = createNarClassLoader(bundleDetail.getWorkingDirectory(), narDependencyClassLoader, logDetails);
                         }
                     }
                 }
@@ -472,10 +472,16 @@ public final class NarClassLoaders {
      * @throws IOException ioe
      * @throws ClassNotFoundException cfne
      */
-    private static ClassLoader createNarClassLoader(final File narDirectory, final ClassLoader parentClassLoader) throws IOException, ClassNotFoundException {
+    private static ClassLoader createNarClassLoader(final File narDirectory, final ClassLoader parentClassLoader, final boolean log) throws IOException, ClassNotFoundException {
         logger.debug("Loading NAR file: " + narDirectory.getAbsolutePath());
         final ClassLoader narClassLoader = new NarClassLoader(narDirectory, parentClassLoader);
-        logger.info("Loaded NAR file: " + narDirectory.getAbsolutePath() + " as class loader " + narClassLoader);
+
+        if (log) {
+            logger.info("Loaded NAR file: {} as class loader {}", narDirectory.getAbsolutePath(), narClassLoader);
+        } else {
+            logger.debug("Loaded NAR file: {} as class loader {}", narDirectory.getAbsolutePath(), narClassLoader);
+        }
+
         return narClassLoader;
     }
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarUnpacker.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarUnpacker.java
index bce352f..fbd9991 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarUnpacker.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarUnpacker.java
@@ -258,7 +258,9 @@ public final class NarUnpacker {
 
             final File bundledDependencies = new File(unpackedNar, BUNDLED_DEPENDENCIES_DIRECTORY);
 
-            unpackBundleDocs(docsDirectory, mapping, bundleCoordinate, bundledDependencies);
+            if (docsDirectory != null) {
+                unpackBundleDocs(docsDirectory, mapping, bundleCoordinate, bundledDependencies);
+            }
         }
     }
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/logback.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/logback.xml
index 9022d22..93a9afa 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/logback.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/logback.xml
@@ -97,6 +97,7 @@
     <logger name="org.apache.zookeeper.server.quorum" level="ERROR" />
     <logger name="org.apache.zookeeper.ZooKeeper" level="ERROR" />
     <logger name="org.apache.zookeeper.server.PrepRequestProcessor" level="ERROR" />
+    <logger name="org.apache.nifi.controller.reporting.LogComponentStatuses" level="ERROR" />
 
     <logger name="org.apache.calcite.runtime.CalciteException" level="OFF" />
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/main/java/org/apache/nifi/NiFi.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/main/java/org/apache/nifi/NiFi.java
index 01e1c71..eef3310 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/main/java/org/apache/nifi/NiFi.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/main/java/org/apache/nifi/NiFi.java
@@ -142,8 +142,7 @@ public class NiFi implements NiFiEntryPoint {
         // load the extensions classloaders
         NarClassLoaders narClassLoaders = NarClassLoadersHolder.getInstance();
 
-        narClassLoaders.init(rootClassLoader,
-                properties.getFrameworkWorkingDirectory(), properties.getExtensionsWorkingDirectory());
+        narClassLoaders.init(rootClassLoader, properties.getFrameworkWorkingDirectory(), properties.getExtensionsWorkingDirectory(), true);
 
         // load the framework classloader
         final ClassLoader frameworkClassLoader = narClassLoaders.getFrameworkBundle().getClassLoader();
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/java/org/apache/nifi/processors/hive/TestPutHive3Streaming.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/java/org/apache/nifi/processors/hive/TestPutHive3Streaming.java
index 0db3cad..4c60a20 100644
--- a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/java/org/apache/nifi/processors/hive/TestPutHive3Streaming.java
+++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/java/org/apache/nifi/processors/hive/TestPutHive3Streaming.java
@@ -104,14 +104,10 @@ import java.util.function.BiFunction;
 import static org.apache.nifi.processors.hive.AbstractHive3QLProcessor.ATTR_OUTPUT_TABLES;
 import static org.apache.nifi.processors.hive.PutHive3Streaming.HIVE_STREAMING_RECORD_COUNT_ATTR;
 import static org.apache.nifi.processors.hive.PutHive3Streaming.KERBEROS_CREDENTIALS_SERVICE;
-import static org.hamcrest.CoreMatchers.containsString;
-import static org.hamcrest.CoreMatchers.hasItem;
-import static org.hamcrest.Matchers.hasProperty;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 import static org.mockito.ArgumentMatchers.any;
@@ -390,10 +386,7 @@ public class TestPutHive3Streaming {
         runner.run();
 
         runner.assertTransferCount(PutHive3Streaming.REL_FAILURE, 1);
-        assertThat(
-                runner.getLogger().getErrorMessages(),
-                hasItem(hasProperty("msg", containsString("Exception while trying to stream {} to hive - routing to failure")))
-        );
+        assertFalse(runner.getLogger().getErrorMessages().isEmpty());
     }
 
     @Test
@@ -427,10 +420,7 @@ public class TestPutHive3Streaming {
         runner.run();
 
         runner.assertTransferCount(PutHive3Streaming.REL_FAILURE, 1);
-        assertThat(
-                runner.getLogger().getErrorMessages(),
-                hasItem(hasProperty("msg", containsString("Failed to create {} for {} - routing to failure")))
-        );
+        assertFalse(runner.getLogger().getErrorMessages().isEmpty());
     }
 
     @Test
@@ -501,10 +491,7 @@ public class TestPutHive3Streaming {
         runner.assertTransferCount(PutHive3Streaming.REL_SUCCESS, 0);
         runner.assertTransferCount(PutHive3Streaming.REL_FAILURE, 1);
         runner.assertTransferCount(PutHive3Streaming.REL_RETRY, 0);
-        assertThat(
-                runner.getLogger().getErrorMessages(),
-                hasItem(hasProperty("msg", containsString("Exception while processing {} - routing to failure")))
-        );
+        assertFalse(runner.getLogger().getErrorMessages().isEmpty());
     }
 
     @Test
@@ -617,10 +604,7 @@ public class TestPutHive3Streaming {
         final MockFlowFile flowFile = runner.getFlowFilesForRelationship(PutHive3Streaming.REL_FAILURE).get(0);
         assertEquals("0", flowFile.getAttribute(HIVE_STREAMING_RECORD_COUNT_ATTR));
         assertEquals("default.users", flowFile.getAttribute(ATTR_OUTPUT_TABLES));
-        assertThat(
-                runner.getLogger().getErrorMessages(),
-                hasItem(hasProperty("msg", containsString("Exception while processing {} - routing to failure")))
-        );
+        assertFalse(runner.getLogger().getErrorMessages().isEmpty());
     }
 
     @Test
@@ -674,10 +658,7 @@ public class TestPutHive3Streaming {
 
         runner.assertTransferCount(PutHive3Streaming.REL_SUCCESS, 0);
         runner.assertTransferCount(PutHive3Streaming.REL_FAILURE, 1);
-        assertThat(
-                runner.getLogger().getErrorMessages(),
-                hasItem(hasProperty("msg", containsString("Exception while processing {} - routing to failure")))
-        );
+        assertFalse(runner.getLogger().getErrorMessages().isEmpty());
     }
 
     @Test
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaRecord_2_6.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaRecord_2_6.java
index 13fd811..7f1f032 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaRecord_2_6.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafkaRecord_2_6.java
@@ -158,6 +158,17 @@ public class ConsumeKafkaRecord_2_6 extends AbstractProcessor implements Verifia
             .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
             .build();
 
+    static final PropertyDescriptor COMMIT_OFFSETS = new Builder()
+            .name("Commit Offsets")
+            .displayName("Commit Offsets")
+            .description("Specifies whether or not this Processor should commit the offsets to Kafka after receiving messages. This value should be false when a PublishKafkaRecord processor is " +
+                "expected to commit the offsets using Exactly Once semantics, and should be reserved for dataflows that are designed to run within Stateless NiFi. See Processor's Usage / " +
+                "Additional Details for more information. Note that setting this value to false can lead to significant data duplication or potentially even data loss if the dataflow " +
+                "is not properly configured.")
+            .allowableValues("true", "false")
+            .defaultValue("true")
+            .build();
+
     static final PropertyDescriptor MAX_UNCOMMITTED_TIME = new Builder()
             .name("max-uncommit-offset-wait")
             .displayName("Max Uncommitted Time")
@@ -170,6 +181,7 @@ public class ConsumeKafkaRecord_2_6 extends AbstractProcessor implements Verifia
             .required(false)
             .defaultValue("1 secs")
             .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .dependsOn(COMMIT_OFFSETS, "true")
             .build();
     static final PropertyDescriptor COMMS_TIMEOUT = new Builder()
         .name("Communications Timeout")
@@ -249,12 +261,15 @@ public class ConsumeKafkaRecord_2_6 extends AbstractProcessor implements Verifia
     private final Set<ConsumerLease> activeLeases = Collections.synchronizedSet(new HashSet<>());
 
     static {
-        List<PropertyDescriptor> descriptors = new ArrayList<>();
+        final List<PropertyDescriptor> descriptors = new ArrayList<>();
         descriptors.add(KafkaProcessorUtils.BOOTSTRAP_SERVERS);
         descriptors.add(TOPICS);
         descriptors.add(TOPIC_TYPE);
         descriptors.add(RECORD_READER);
         descriptors.add(RECORD_WRITER);
+        descriptors.add(GROUP_ID);
+        descriptors.add(COMMIT_OFFSETS);
+        descriptors.add(MAX_UNCOMMITTED_TIME);
         descriptors.add(HONOR_TRANSACTIONS);
         descriptors.add(KafkaProcessorUtils.SECURITY_PROTOCOL);
         descriptors.add(KafkaProcessorUtils.SASL_MECHANISM);
@@ -266,14 +281,12 @@ public class ConsumeKafkaRecord_2_6 extends AbstractProcessor implements Verifia
         descriptors.add(KafkaProcessorUtils.PASSWORD);
         descriptors.add(KafkaProcessorUtils.TOKEN_AUTH);
         descriptors.add(KafkaProcessorUtils.SSL_CONTEXT_SERVICE);
-        descriptors.add(GROUP_ID);
         descriptors.add(SEPARATE_BY_KEY);
         descriptors.add(KEY_ATTRIBUTE_ENCODING);
         descriptors.add(AUTO_OFFSET_RESET);
         descriptors.add(MESSAGE_HEADER_ENCODING);
         descriptors.add(HEADER_NAME_REGEX);
         descriptors.add(MAX_POLL_RECORDS);
-        descriptors.add(MAX_UNCOMMITTED_TIME);
         descriptors.add(COMMS_TIMEOUT);
         DESCRIPTORS = Collections.unmodifiableList(descriptors);
 
@@ -368,7 +381,8 @@ public class ConsumeKafkaRecord_2_6 extends AbstractProcessor implements Verifia
 
     protected ConsumerPool createConsumerPool(final ProcessContext context, final ComponentLog log) {
         final int maxLeases = context.getMaxConcurrentTasks();
-        final long maxUncommittedTime = context.getProperty(MAX_UNCOMMITTED_TIME).asTimePeriod(TimeUnit.MILLISECONDS);
+        final Long maxUncommittedTime = context.getProperty(MAX_UNCOMMITTED_TIME).asTimePeriod(TimeUnit.MILLISECONDS);
+        final boolean commitOffsets = context.getProperty(COMMIT_OFFSETS).asBoolean();
 
         final Map<String, Object> props = new HashMap<>();
         KafkaProcessorUtils.buildCommonKafkaProperties(context, ConsumerConfig.class, props);
@@ -412,11 +426,11 @@ public class ConsumeKafkaRecord_2_6 extends AbstractProcessor implements Verifia
             }
 
             return new ConsumerPool(maxLeases, readerFactory, writerFactory, props, topics, maxUncommittedTime, securityProtocol,
-                bootstrapServers, log, honorTransactions, charset, headerNamePattern, separateByKey, keyEncoding, partitionsToConsume);
+                bootstrapServers, log, honorTransactions, charset, headerNamePattern, separateByKey, keyEncoding, partitionsToConsume, commitOffsets);
         } else if (topicType.equals(TOPIC_PATTERN.getValue())) {
             final Pattern topicPattern = Pattern.compile(topicListing.trim());
             return new ConsumerPool(maxLeases, readerFactory, writerFactory, props, topicPattern, maxUncommittedTime, securityProtocol,
-                bootstrapServers, log, honorTransactions, charset, headerNamePattern, separateByKey, keyEncoding, partitionsToConsume);
+                bootstrapServers, log, honorTransactions, charset, headerNamePattern, separateByKey, keyEncoding, partitionsToConsume, commitOffsets);
         } else {
             getLogger().error("Subscription type has an unknown value {}", new Object[] {topicType});
             return null;
@@ -473,7 +487,8 @@ public class ConsumeKafkaRecord_2_6 extends AbstractProcessor implements Verifia
                 while (this.isScheduled() && lease.continuePolling()) {
                     lease.poll();
                 }
-                if (this.isScheduled() && !lease.commit()) {
+
+                if (!lease.commit()) {
                     context.yield();
                 }
             } catch (final WakeupException we) {
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka_2_6.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka_2_6.java
index 58f2b24..1dbae80 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka_2_6.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka_2_6.java
@@ -183,6 +183,16 @@ public class ConsumeKafka_2_6 extends AbstractProcessor implements VerifiablePro
             .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
             .build();
 
+    static final PropertyDescriptor COMMIT_OFFSETS = new PropertyDescriptor.Builder()
+            .name("Commit Offsets")
+            .displayName("Commit Offsets")
+            .description("Specifies whether or not this Processor should commit the offsets to Kafka after receiving messages. Typically, we want this value set to true " +
+                "so that messages that are received are not duplicated. However, in certain scenarios, we may want to avoid committing the offsets, that the data can be " +
+                "processed and later acknowledged by PublishKafkaRecord in order to provide Exactly Once semantics. See Processor's Usage / Additional Details for more information.")
+            .allowableValues("true", "false")
+            .defaultValue("true")
+            .build();
+
     static final PropertyDescriptor MAX_UNCOMMITTED_TIME = new PropertyDescriptor.Builder()
             .name("max-uncommit-offset-wait")
             .displayName("Max Uncommitted Time")
@@ -195,7 +205,9 @@ public class ConsumeKafka_2_6 extends AbstractProcessor implements VerifiablePro
             .required(false)
             .defaultValue("1 secs")
             .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .dependsOn(COMMIT_OFFSETS, "true")
             .build();
+
     static final PropertyDescriptor COMMS_TIMEOUT = new PropertyDescriptor.Builder()
         .name("Communications Timeout")
         .displayName("Communications Timeout")
@@ -239,21 +251,34 @@ public class ConsumeKafka_2_6 extends AbstractProcessor implements VerifiablePro
     private final Set<ConsumerLease> activeLeases = Collections.synchronizedSet(new HashSet<>());
 
     static {
-        List<PropertyDescriptor> descriptors = new ArrayList<>();
-        descriptors.addAll(KafkaProcessorUtils.getCommonPropertyDescriptors());
+        final List<PropertyDescriptor> descriptors = new ArrayList<>();
+        descriptors.add(KafkaProcessorUtils.BOOTSTRAP_SERVERS);
         descriptors.add(TOPICS);
         descriptors.add(TOPIC_TYPE);
-        descriptors.add(HONOR_TRANSACTIONS);
         descriptors.add(GROUP_ID);
-        descriptors.add(AUTO_OFFSET_RESET);
-        descriptors.add(KEY_ATTRIBUTE_ENCODING);
+        descriptors.add(COMMIT_OFFSETS);
+        descriptors.add(MAX_UNCOMMITTED_TIME);
+        descriptors.add(HONOR_TRANSACTIONS);
         descriptors.add(MESSAGE_DEMARCATOR);
         descriptors.add(SEPARATE_BY_KEY);
+        descriptors.add(KafkaProcessorUtils.SECURITY_PROTOCOL);
+        descriptors.add(KafkaProcessorUtils.SASL_MECHANISM);
+        descriptors.add(KafkaProcessorUtils.KERBEROS_CREDENTIALS_SERVICE);
+        descriptors.add(KafkaProcessorUtils.SELF_CONTAINED_KERBEROS_USER_SERVICE);
+        descriptors.add(KafkaProcessorUtils.JAAS_SERVICE_NAME);
+        descriptors.add(KafkaProcessorUtils.USER_PRINCIPAL);
+        descriptors.add(KafkaProcessorUtils.USER_KEYTAB);
+        descriptors.add(KafkaProcessorUtils.USERNAME);
+        descriptors.add(KafkaProcessorUtils.PASSWORD);
+        descriptors.add(KafkaProcessorUtils.TOKEN_AUTH);
+        descriptors.add(KafkaProcessorUtils.SSL_CONTEXT_SERVICE);
+        descriptors.add(KEY_ATTRIBUTE_ENCODING);
+        descriptors.add(AUTO_OFFSET_RESET);
         descriptors.add(MESSAGE_HEADER_ENCODING);
         descriptors.add(HEADER_NAME_REGEX);
         descriptors.add(MAX_POLL_RECORDS);
-        descriptors.add(MAX_UNCOMMITTED_TIME);
         descriptors.add(COMMS_TIMEOUT);
+
         DESCRIPTORS = Collections.unmodifiableList(descriptors);
         RELATIONSHIPS = Collections.singleton(REL_SUCCESS);
     }
@@ -342,10 +367,11 @@ public class ConsumeKafka_2_6 extends AbstractProcessor implements VerifiablePro
 
     protected ConsumerPool createConsumerPool(final ProcessContext context, final ComponentLog log) {
         final int maxLeases = context.getMaxConcurrentTasks();
-        final long maxUncommittedTime = context.getProperty(MAX_UNCOMMITTED_TIME).asTimePeriod(TimeUnit.MILLISECONDS);
+        final Long maxUncommittedTime = context.getProperty(MAX_UNCOMMITTED_TIME).asTimePeriod(TimeUnit.MILLISECONDS);
+        final boolean commitOffsets = context.getProperty(COMMIT_OFFSETS).asBoolean();
+
         final byte[] demarcator = context.getProperty(ConsumeKafka_2_6.MESSAGE_DEMARCATOR).isSet()
-                ? context.getProperty(ConsumeKafka_2_6.MESSAGE_DEMARCATOR).evaluateAttributeExpressions().getValue().getBytes(StandardCharsets.UTF_8)
-                : null;
+                ? context.getProperty(ConsumeKafka_2_6.MESSAGE_DEMARCATOR).evaluateAttributeExpressions().getValue().getBytes(StandardCharsets.UTF_8) : null;
         final Map<String, Object> props = new HashMap<>();
         KafkaProcessorUtils.buildCommonKafkaProperties(context, ConsumerConfig.class, props);
         props.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false");
@@ -386,11 +412,11 @@ public class ConsumeKafka_2_6 extends AbstractProcessor implements VerifiablePro
             }
 
             return new ConsumerPool(maxLeases, demarcator, separateByKey, props, topics, maxUncommittedTime, keyEncoding, securityProtocol,
-                bootstrapServers, log, honorTransactions, charset, headerNamePattern, partitionsToConsume);
+                bootstrapServers, log, honorTransactions, charset, headerNamePattern, partitionsToConsume, commitOffsets);
         } else if (topicType.equals(TOPIC_PATTERN.getValue())) {
             final Pattern topicPattern = Pattern.compile(topicListing.trim());
             return new ConsumerPool(maxLeases, demarcator, separateByKey, props, topicPattern, maxUncommittedTime, keyEncoding, securityProtocol,
-                bootstrapServers, log, honorTransactions, charset, headerNamePattern, partitionsToConsume);
+                bootstrapServers, log, honorTransactions, charset, headerNamePattern, partitionsToConsume, commitOffsets);
         } else {
             getLogger().error("Subscription type has an unknown value {}", new Object[] {topicType});
             return null;
@@ -447,7 +473,8 @@ public class ConsumeKafka_2_6 extends AbstractProcessor implements VerifiablePro
                 while (this.isScheduled() && lease.continuePolling()) {
                     lease.poll();
                 }
-                if (this.isScheduled() && !lease.commit()) {
+
+                if (!lease.commit()) {
                     context.yield();
                 }
             } catch (final WakeupException we) {
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerLease.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerLease.java
index bdb9b3d..6323735 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerLease.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerLease.java
@@ -52,10 +52,12 @@ import java.time.Duration;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
+import java.util.Set;
 import java.util.concurrent.TimeUnit;
 import java.util.regex.Pattern;
 import java.util.stream.Collectors;
@@ -74,7 +76,7 @@ import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.UTF8_E
  */
 public abstract class ConsumerLease implements Closeable, ConsumerRebalanceListener {
 
-    private final long maxWaitMillis;
+    private final Long maxWaitMillis;
     private final Consumer<byte[], byte[]> kafkaConsumer;
     private final ComponentLog logger;
     private final byte[] demarcatorBytes;
@@ -86,6 +88,7 @@ public abstract class ConsumerLease implements Closeable, ConsumerRebalanceListe
     private final Charset headerCharacterSet;
     private final Pattern headerNamePattern;
     private final boolean separateByKey;
+    private final boolean commitOffsets;
     private boolean poisoned = false;
     //used for tracking demarcated flowfiles to their TopicPartition so we can append
     //to them on subsequent poll calls
@@ -96,7 +99,7 @@ public abstract class ConsumerLease implements Closeable, ConsumerRebalanceListe
     private int totalMessages = 0;
 
     ConsumerLease(
-            final long maxWaitMillis,
+            final Long maxWaitMillis,
             final Consumer<byte[], byte[]> kafkaConsumer,
             final byte[] demarcatorBytes,
             final String keyEncoding,
@@ -107,7 +110,8 @@ public abstract class ConsumerLease implements Closeable, ConsumerRebalanceListe
             final ComponentLog logger,
             final Charset headerCharacterSet,
             final Pattern headerNamePattern,
-            final boolean separateByKey) {
+            final boolean separateByKey,
+            final boolean commitMessageOffsets) {
         this.maxWaitMillis = maxWaitMillis;
         this.kafkaConsumer = kafkaConsumer;
         this.demarcatorBytes = demarcatorBytes;
@@ -120,6 +124,7 @@ public abstract class ConsumerLease implements Closeable, ConsumerRebalanceListe
         this.headerCharacterSet = headerCharacterSet;
         this.headerNamePattern = headerNamePattern;
         this.separateByKey = separateByKey;
+        this.commitOffsets = commitMessageOffsets;
     }
 
     /**
@@ -148,7 +153,6 @@ public abstract class ConsumerLease implements Closeable, ConsumerRebalanceListe
     @Override
     public void onPartitionsRevoked(final Collection<TopicPartition> partitions) {
         logger.debug("Rebalance Alert: Partitions '{}' revoked for lease '{}' with consumer '{}'", new Object[]{partitions, this, kafkaConsumer});
-        //force a commit here.  Can reuse the session and consumer after this but must commit now to avoid duplicates if kafka reassigns partition
         commit();
     }
 
@@ -173,7 +177,7 @@ public abstract class ConsumerLease implements Closeable, ConsumerRebalanceListe
      * flowfiles necessary or appends to existing ones if in demarcation mode.
      */
     void poll() {
-        /**
+        /*
          * Implementation note:
          * Even if ConsumeKafka is not scheduled to poll due to downstream connection back-pressure is engaged,
          * for longer than session.timeout.ms (defaults to 10 sec), Kafka consumer sends heartbeat from background thread.
@@ -194,6 +198,15 @@ public abstract class ConsumerLease implements Closeable, ConsumerRebalanceListe
         }
     }
 
+    void abort() {
+        rollback(kafkaConsumer.assignment());
+        final ProcessSession session = getProcessSession();
+        if (session != null) {
+            session.rollback();
+        }
+        resetInternalState();
+    }
+
     /**
      * Notifies Kafka to commit the offsets for the specified topic/partition
      * pairs to the specified offsets w/the given metadata. This can offer
@@ -210,8 +223,15 @@ public abstract class ConsumerLease implements Closeable, ConsumerRebalanceListe
             resetInternalState();
             return false;
         }
+
+        if (isPoisoned()) {
+            // Failed to commit the session. Rollback the offsets.
+            abort();
+            return false;
+        }
+
         try {
-            /**
+            /*
              * Committing the nifi session then the offsets means we have an at
              * least once guarantee here. If we reversed the order we'd have at
              * most once.
@@ -219,12 +239,29 @@ public abstract class ConsumerLease implements Closeable, ConsumerRebalanceListe
             final Collection<FlowFile> bundledFlowFiles = getBundles();
             if (!bundledFlowFiles.isEmpty()) {
                 getProcessSession().transfer(bundledFlowFiles, REL_SUCCESS);
+
+                if (logger.isDebugEnabled()) {
+                    for (final FlowFile flowFile : bundledFlowFiles) {
+                        final String recordCountAttribute = flowFile.getAttribute("record.count");
+                        final String recordCount = recordCountAttribute == null ? "1" : recordCountAttribute;
+                        logger.debug("Transferred {} with {} records, max offset of {}", flowFile, recordCount, flowFile.getAttribute(KafkaProcessorUtils.KAFKA_MAX_OFFSET));
+                    }
+                }
             }
 
+            final Map<TopicPartition, OffsetAndMetadata> offsetsMap = new HashMap<>(uncommittedOffsetsMap);
+            final Set<TopicPartition> assignedPartitions = kafkaConsumer.assignment();
+
             getProcessSession().commitAsync(() -> {
-                final Map<TopicPartition, OffsetAndMetadata> offsetsMap = uncommittedOffsetsMap;
-                kafkaConsumer.commitSync(offsetsMap);
+                if (commitOffsets) {
+                    kafkaConsumer.commitSync(offsetsMap);
+                }
+                resetInternalState();
+            }, failureCause -> {
+                // Failed to commit the session. Rollback the offsets.
+                logger.error("Failed to commit ProcessSession after consuming records from Kafka. Will rollback Kafka Offsets", failureCause);
                 resetInternalState();
+                rollback(assignedPartitions);
             });
 
             return true;
@@ -271,7 +308,7 @@ public abstract class ConsumerLease implements Closeable, ConsumerRebalanceListe
             leaseStartNanos = System.nanoTime();
         }
         final long durationMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - leaseStartNanos);
-        if (durationMillis > maxWaitMillis) {
+        if (maxWaitMillis == null || durationMillis > maxWaitMillis) {
             return false;
         }
 
@@ -326,12 +363,12 @@ public abstract class ConsumerLease implements Closeable, ConsumerRebalanceListe
     public abstract void yield();
 
     private void processRecords(final ConsumerRecords<byte[], byte[]> records) {
-        records.partitions().stream().forEach(partition -> {
+        records.partitions().forEach(partition -> {
             List<ConsumerRecord<byte[], byte[]>> messages = records.records(partition);
             if (!messages.isEmpty()) {
                 //update maximum offset map for this topic partition
                 long maxOffset = messages.stream()
-                        .mapToLong(record -> record.offset())
+                        .mapToLong(ConsumerRecord::offset)
                         .max()
                         .getAsLong();
 
@@ -341,7 +378,7 @@ public abstract class ConsumerLease implements Closeable, ConsumerRebalanceListe
                 } else if (readerFactory != null && writerFactory != null) {
                     writeRecordData(getProcessSession(), messages, partition);
                 } else {
-                    messages.stream().forEach(message -> {
+                    messages.forEach(message -> {
                         writeData(getProcessSession(), message, partition);
                     });
                 }
@@ -407,7 +444,7 @@ public abstract class ConsumerLease implements Closeable, ConsumerRebalanceListe
     private void writeData(final ProcessSession session, ConsumerRecord<byte[], byte[]> record, final TopicPartition topicPartition) {
         FlowFile flowFile = session.create();
         final BundleTracker tracker = new BundleTracker(record, topicPartition, keyEncoding);
-        tracker.incrementRecordCount(1);
+        tracker.incrementRecordCount(1, record.offset(), record.leaderEpoch().orElse(null));
         final byte[] value = record.value();
         if (value != null) {
             flowFile = session.write(flowFile, out -> {
@@ -445,7 +482,16 @@ public abstract class ConsumerLease implements Closeable, ConsumerRebalanceListe
             }
             flowFile = tracker.flowFile;
 
-            tracker.incrementRecordCount(recordList.size());
+            // Determine max offset of any record to provide to the MessageTracker.
+            long maxOffset = recordList.get(0).offset();
+            int leaderEpoch = -1;
+            for (final ConsumerRecord<byte[], byte[]> record : recordList) {
+                maxOffset = Math.max(maxOffset, record.offset());
+                leaderEpoch = Math.max(record.leaderEpoch().orElse(leaderEpoch), leaderEpoch);
+            }
+
+            tracker.incrementRecordCount(recordList.size(), maxOffset, leaderEpoch >= 0 ? leaderEpoch : null);
+
             flowFile = session.append(flowFile, out -> {
                 boolean useDemarcator = demarcateFirstRecord;
                 for (final ConsumerRecord<byte[], byte[]> record : recordList) {
@@ -588,7 +634,7 @@ public abstract class ConsumerLease implements Closeable, ConsumerRebalanceListe
                                 continue;
                             }
 
-                            tracker.incrementRecordCount(1L);
+                            tracker.incrementRecordCount(1L, consumerRecord.offset(), consumerRecord.leaderEpoch().orElse(null));
                             session.adjustCounter("Records Received", 1L, false);
                         }
                     } catch (final IOException | MalformedRecordException | SchemaValidationException e) {
@@ -618,25 +664,42 @@ public abstract class ConsumerLease implements Closeable, ConsumerRebalanceListe
     }
 
     private void rollback(final TopicPartition topicPartition) {
+        rollback(Collections.singleton(topicPartition));
+    }
+
+    private void rollback(final Set<TopicPartition> topicPartitions) {
         try {
-            OffsetAndMetadata offsetAndMetadata = uncommittedOffsetsMap.get(topicPartition);
-            if (offsetAndMetadata == null) {
-                offsetAndMetadata = kafkaConsumer.committed(topicPartition);
+            final Map<TopicPartition, OffsetAndMetadata> metadataMap = kafkaConsumer.committed(topicPartitions);
+            for (final Map.Entry<TopicPartition, OffsetAndMetadata> entry : metadataMap.entrySet()) {
+                final TopicPartition topicPartition = entry.getKey();
+                final OffsetAndMetadata offsetAndMetadata = entry.getValue();
+
+                if (offsetAndMetadata == null) {
+                    kafkaConsumer.seekToBeginning(Collections.singleton(topicPartition));
+                    logger.info("Rolling back offsets so that {}-{} it is at the beginning", topicPartition.topic(), topicPartition.partition());
+                } else {
+                    kafkaConsumer.seek(topicPartition, offsetAndMetadata.offset());
+                    logger.info("Rolling back offsets so that {}-{} has offset of {}", topicPartition.topic(), topicPartition.partition(), offsetAndMetadata.offset());
+                }
             }
-
-            final long offset = offsetAndMetadata == null ? 0L : offsetAndMetadata.offset();
-            kafkaConsumer.seek(topicPartition, offset);
         } catch (final Exception rollbackException) {
             logger.warn("Attempted to rollback Kafka message offset but was unable to do so", rollbackException);
+            poison();
         }
     }
 
 
-
     private void populateAttributes(final BundleTracker tracker) {
         final Map<String, String> kafkaAttrs = new HashMap<>();
         kafkaAttrs.put(KafkaProcessorUtils.KAFKA_OFFSET, String.valueOf(tracker.initialOffset));
         kafkaAttrs.put(KafkaProcessorUtils.KAFKA_TIMESTAMP, String.valueOf(tracker.initialTimestamp));
+        kafkaAttrs.put(KafkaProcessorUtils.KAFKA_MAX_OFFSET, String.valueOf(tracker.maxOffset));
+        if (tracker.leaderEpoch != null) {
+            kafkaAttrs.put(KafkaProcessorUtils.KAFKA_LEADER_EPOCH, String.valueOf(tracker.leaderEpoch));
+        }
+
+        kafkaAttrs.put(KafkaProcessorUtils.KAFKA_CONSUMER_GROUP_ID, kafkaConsumer.groupMetadata().groupId());
+        kafkaAttrs.put(KafkaProcessorUtils.KAFKA_CONSUMER_OFFSETS_COMMITTED, String.valueOf(commitOffsets));
 
         // If we have a kafka key, we will add it as an attribute only if
         // the FlowFile contains a single Record, or if the Records have been separated by Key,
@@ -658,6 +721,7 @@ public abstract class ConsumerLease implements Closeable, ConsumerRebalanceListe
                 kafkaAttrs.put("record.count", String.valueOf(tracker.totalRecords));
             }
         }
+
         final FlowFile newFlowFile = getProcessSession().putAllAttributes(tracker.flowFile, kafkaAttrs);
         final long executionDurationMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - leaseStartNanos);
         final String transitUri = KafkaProcessorUtils.buildTransitURI(securityProtocol, bootstrapServers, tracker.topic);
@@ -675,6 +739,8 @@ public abstract class ConsumerLease implements Closeable, ConsumerRebalanceListe
         final RecordSetWriter recordWriter;
         FlowFile flowFile;
         long totalRecords = 0;
+        long maxOffset;
+        Integer leaderEpoch;
 
         private BundleTracker(final ConsumerRecord<byte[], byte[]> initialRecord, final TopicPartition topicPartition, final String keyEncoding) {
             this(initialRecord, topicPartition, keyEncoding, null);
@@ -682,15 +748,21 @@ public abstract class ConsumerLease implements Closeable, ConsumerRebalanceListe
 
         private BundleTracker(final ConsumerRecord<byte[], byte[]> initialRecord, final TopicPartition topicPartition, final String keyEncoding, final RecordSetWriter recordWriter) {
             this.initialOffset = initialRecord.offset();
+            this.maxOffset = initialOffset;
             this.initialTimestamp = initialRecord.timestamp();
             this.partition = topicPartition.partition();
             this.topic = topicPartition.topic();
             this.recordWriter = recordWriter;
             this.key = encodeKafkaKey(initialRecord.key(), keyEncoding);
+            this.leaderEpoch = initialRecord.leaderEpoch().orElse(null);
         }
 
-        private void incrementRecordCount(final long count) {
+        private void incrementRecordCount(final long count, final long maxOffset, final Integer leaderEpoch) {
             totalRecords += count;
+            this.maxOffset = Math.max(this.maxOffset, maxOffset);
+            if (leaderEpoch != null) {
+                this.leaderEpoch = (this.leaderEpoch == null) ? leaderEpoch : Math.max(this.leaderEpoch, leaderEpoch);
+            }
         }
 
         private void updateFlowFile(final FlowFile flowFile) {
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerPool.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerPool.java
index 381c7b1..e10a75b 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerPool.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerPool.java
@@ -62,7 +62,7 @@ public class ConsumerPool implements Closeable {
     private final List<String> topics;
     private final Pattern topicPattern;
     private final Map<String, Object> kafkaProperties;
-    private final long maxWaitMillis;
+    private final Long maxWaitMillis;
     private final ComponentLog logger;
     private final byte[] demarcatorBytes;
     private final String keyEncoding;
@@ -75,6 +75,7 @@ public class ConsumerPool implements Closeable {
     private final Pattern headerNamePattern;
     private final boolean separateByKey;
     private final int[] partitionsToConsume;
+    private final boolean commitOffsets;
     private final AtomicLong consumerCreatedCountRef = new AtomicLong();
     private final AtomicLong consumerClosedCountRef = new AtomicLong();
     private final AtomicLong leasesObtainedCountRef = new AtomicLong();
@@ -106,7 +107,7 @@ public class ConsumerPool implements Closeable {
             final boolean separateByKey,
             final Map<String, Object> kafkaProperties,
             final List<String> topics,
-            final long maxWaitMillis,
+            final Long maxWaitMillis,
             final String keyEncoding,
             final String securityProtocol,
             final String bootstrapServers,
@@ -114,7 +115,8 @@ public class ConsumerPool implements Closeable {
             final boolean honorTransactions,
             final Charset headerCharacterSet,
             final Pattern headerNamePattern,
-            final int[] partitionsToConsume) {
+            final int[] partitionsToConsume,
+            final boolean commitOffsets) {
         this.pooledLeases = new LinkedBlockingQueue<>();
         this.maxWaitMillis = maxWaitMillis;
         this.logger = logger;
@@ -132,6 +134,7 @@ public class ConsumerPool implements Closeable {
         this.headerNamePattern = headerNamePattern;
         this.separateByKey = separateByKey;
         this.partitionsToConsume = partitionsToConsume;
+        this.commitOffsets = commitOffsets;
         enqueueAssignedPartitions(partitionsToConsume);
     }
 
@@ -141,7 +144,7 @@ public class ConsumerPool implements Closeable {
             final boolean separateByKey,
             final Map<String, Object> kafkaProperties,
             final Pattern topics,
-            final long maxWaitMillis,
+            final Long maxWaitMillis,
             final String keyEncoding,
             final String securityProtocol,
             final String bootstrapServers,
@@ -149,7 +152,8 @@ public class ConsumerPool implements Closeable {
             final boolean honorTransactions,
             final Charset headerCharacterSet,
             final Pattern headerNamePattern,
-            final int[] partitionsToConsume) {
+            final int[] partitionsToConsume,
+            final boolean commitOffsets) {
         this.pooledLeases = new LinkedBlockingQueue<>();
         this.maxWaitMillis = maxWaitMillis;
         this.logger = logger;
@@ -167,6 +171,7 @@ public class ConsumerPool implements Closeable {
         this.headerNamePattern = headerNamePattern;
         this.separateByKey = separateByKey;
         this.partitionsToConsume = partitionsToConsume;
+        this.commitOffsets = commitOffsets;
         enqueueAssignedPartitions(partitionsToConsume);
     }
 
@@ -176,7 +181,7 @@ public class ConsumerPool implements Closeable {
             final RecordSetWriterFactory writerFactory,
             final Map<String, Object> kafkaProperties,
             final Pattern topics,
-            final long maxWaitMillis,
+            final Long maxWaitMillis,
             final String securityProtocol,
             final String bootstrapServers,
             final ComponentLog logger,
@@ -185,7 +190,8 @@ public class ConsumerPool implements Closeable {
             final Pattern headerNamePattern,
             final boolean separateByKey,
             final String keyEncoding,
-            final int[] partitionsToConsume) {
+            final int[] partitionsToConsume,
+            final boolean commitOffsets) {
         this.pooledLeases = new LinkedBlockingQueue<>();
         this.maxWaitMillis = maxWaitMillis;
         this.logger = logger;
@@ -203,6 +209,7 @@ public class ConsumerPool implements Closeable {
         this.separateByKey = separateByKey;
         this.keyEncoding = keyEncoding;
         this.partitionsToConsume = partitionsToConsume;
+        this.commitOffsets = commitOffsets;
         enqueueAssignedPartitions(partitionsToConsume);
     }
 
@@ -212,7 +219,7 @@ public class ConsumerPool implements Closeable {
             final RecordSetWriterFactory writerFactory,
             final Map<String, Object> kafkaProperties,
             final List<String> topics,
-            final long maxWaitMillis,
+            final Long maxWaitMillis,
             final String securityProtocol,
             final String bootstrapServers,
             final ComponentLog logger,
@@ -221,7 +228,8 @@ public class ConsumerPool implements Closeable {
             final Pattern headerNamePattern,
             final boolean separateByKey,
             final String keyEncoding,
-            final int[] partitionsToConsume) {
+            final int[] partitionsToConsume,
+            final boolean commitOffsets) {
         this.pooledLeases = new LinkedBlockingQueue<>();
         this.maxWaitMillis = maxWaitMillis;
         this.logger = logger;
@@ -239,6 +247,7 @@ public class ConsumerPool implements Closeable {
         this.separateByKey = separateByKey;
         this.keyEncoding = keyEncoding;
         this.partitionsToConsume = partitionsToConsume;
+        this.commitOffsets = commitOffsets;
         enqueueAssignedPartitions(partitionsToConsume);
     }
 
@@ -552,7 +561,7 @@ public class ConsumerPool implements Closeable {
     public void close() {
         final List<SimpleConsumerLease> leases = new ArrayList<>();
         pooledLeases.drainTo(leases);
-        leases.stream().forEach((lease) -> {
+        leases.forEach((lease) -> {
             lease.close(true);
         });
     }
@@ -585,7 +594,7 @@ public class ConsumerPool implements Closeable {
 
         private SimpleConsumerLease(final Consumer<byte[], byte[]> consumer, final List<TopicPartition> assignedPartitions) {
             super(maxWaitMillis, consumer, demarcatorBytes, keyEncoding, securityProtocol, bootstrapServers,
-                readerFactory, writerFactory, logger, headerCharacterSet, headerNamePattern, separateByKey);
+                readerFactory, writerFactory, logger, headerCharacterSet, headerNamePattern, separateByKey, commitOffsets);
             this.consumer = consumer;
             this.assignedPartitions = assignedPartitions;
         }
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaProcessorUtils.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaProcessorUtils.java
index c1f625b..6dbacd2 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaProcessorUtils.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/KafkaProcessorUtils.java
@@ -47,7 +47,6 @@ import javax.security.auth.login.AppConfigurationEntry;
 import java.lang.reflect.Field;
 import java.lang.reflect.Modifier;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Collection;
 import java.util.HashSet;
 import java.util.List;
@@ -75,8 +74,12 @@ public final class KafkaProcessorUtils {
     static final String KAFKA_TOPIC = "kafka.topic";
     static final String KAFKA_PARTITION = "kafka.partition";
     static final String KAFKA_OFFSET = "kafka.offset";
+    static final String KAFKA_MAX_OFFSET = "kafka.max.offset";
+    static final String KAFKA_LEADER_EPOCH = "kafka.leader.epoch";
     static final String KAFKA_TIMESTAMP = "kafka.timestamp";
     static final String KAFKA_COUNT = "kafka.count";
+    static final String KAFKA_CONSUMER_GROUP_ID = "kafka.consumer.id";
+    static final String KAFKA_CONSUMER_OFFSETS_COMMITTED = "kafka.consumer.offsets.committed";
 
     static final AllowableValue SEC_PLAINTEXT = new AllowableValue("PLAINTEXT", "PLAINTEXT", "PLAINTEXT");
     static final AllowableValue SEC_SSL = new AllowableValue("SSL", "SSL", "SSL");
@@ -222,22 +225,6 @@ public final class KafkaProcessorUtils {
 
     public static final String JAVA_SECURITY_AUTH_LOGIN_CONFIG = "java.security.auth.login.config";
 
-    static List<PropertyDescriptor> getCommonPropertyDescriptors() {
-        return Arrays.asList(
-                BOOTSTRAP_SERVERS,
-                SECURITY_PROTOCOL,
-                SASL_MECHANISM,
-                JAAS_SERVICE_NAME,
-                KERBEROS_CREDENTIALS_SERVICE,
-                SELF_CONTAINED_KERBEROS_USER_SERVICE,
-                USER_PRINCIPAL,
-                USER_KEYTAB,
-                USERNAME,
-                PASSWORD,
-                TOKEN_AUTH,
-                SSL_CONTEXT_SERVICE
-        );
-    }
 
     public static Collection<ValidationResult> validateCommonProperties(final ValidationContext validationContext) {
         List<ValidationResult> results = new ArrayList<>();
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaRecord_2_6.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaRecord_2_6.java
index 6cb3e4c..8dbbdb9 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaRecord_2_6.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaRecord_2_6.java
@@ -30,6 +30,7 @@ import org.apache.nifi.annotation.documentation.SeeAlso;
 import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.annotation.lifecycle.OnStopped;
 import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.ConfigVerificationResult;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.PropertyDescriptor.Builder;
 import org.apache.nifi.components.ValidationContext;
@@ -37,7 +38,6 @@ import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.AbstractProcessor;
-import org.apache.nifi.components.ConfigVerificationResult;
 import org.apache.nifi.processor.DataUnit;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSession;
@@ -45,7 +45,6 @@ import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.processor.VerifiableProcessor;
 import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.io.InputStreamCallback;
-import org.apache.nifi.processor.util.FlowFileFilters;
 import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.record.path.RecordPath;
 import org.apache.nifi.record.path.RecordPathResult;
@@ -259,6 +258,7 @@ public class PublishKafkaRecord_2_6 extends AbstractProcessor implements Verifia
         .description("When Use Transaction is set to true, KafkaProducer config 'transactional.id' will be a generated UUID and will be prefixed with this string.")
         .expressionLanguageSupported(VARIABLE_REGISTRY)
         .addValidator(StandardValidators.NON_EMPTY_EL_VALIDATOR)
+        .dependsOn(USE_TRANSACTIONS, "true")
         .required(false)
         .build();
     static final PropertyDescriptor MESSAGE_HEADER_ENCODING = new Builder()
@@ -294,14 +294,15 @@ public class PublishKafkaRecord_2_6 extends AbstractProcessor implements Verifia
         properties.add(RECORD_READER);
         properties.add(RECORD_WRITER);
         properties.add(USE_TRANSACTIONS);
-        properties.add(KafkaProcessorUtils.FAILURE_STRATEGY);
         properties.add(TRANSACTIONAL_ID_PREFIX);
+        properties.add(KafkaProcessorUtils.FAILURE_STRATEGY);
         properties.add(DELIVERY_GUARANTEE);
         properties.add(ATTRIBUTE_NAME_REGEX);
         properties.add(MESSAGE_HEADER_ENCODING);
         properties.add(KafkaProcessorUtils.SECURITY_PROTOCOL);
         properties.add(KafkaProcessorUtils.SASL_MECHANISM);
         properties.add(KafkaProcessorUtils.KERBEROS_CREDENTIALS_SERVICE);
+        properties.add(KafkaProcessorUtils.SELF_CONTAINED_KERBEROS_USER_SERVICE);
         properties.add(KafkaProcessorUtils.JAAS_SERVICE_NAME);
         properties.add(KafkaProcessorUtils.USER_PRINCIPAL);
         properties.add(KafkaProcessorUtils.USER_KEYTAB);
@@ -435,7 +436,7 @@ public class PublishKafkaRecord_2_6 extends AbstractProcessor implements Verifia
 
     @Override
     public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
-        final List<FlowFile> flowFiles = session.get(FlowFileFilters.newSizeBasedFilter(1, DataUnit.MB, 500));
+        final List<FlowFile> flowFiles = PublishKafkaUtil.pollFlowFiles(session);
         if (flowFiles.isEmpty()) {
             return;
         }
@@ -498,6 +499,11 @@ public class PublishKafkaRecord_2_6 extends AbstractProcessor implements Verifia
                                 }
                             }
                         });
+
+                        // If consumer offsets haven't been committed, add them to the transaction.
+                        if (useTransactions && "false".equals(flowFile.getAttribute(KafkaProcessorUtils.KAFKA_CONSUMER_OFFSETS_COMMITTED))) {
+                            PublishKafkaUtil.addConsumerOffsets(lease, flowFile, getLogger());
+                        }
                     } catch (final Exception e) {
                         // The FlowFile will be obtained and the error logged below, when calling publishResult.getFailedFlowFiles()
                         lease.fail(flowFile, e);
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaUtil.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaUtil.java
new file mode 100644
index 0000000..20dc112
--- /dev/null
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaUtil.java
@@ -0,0 +1,124 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.processors.kafka.pubsub;
+
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.DataUnit;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.util.FlowFileFilters;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.KAFKA_CONSUMER_GROUP_ID;
+import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.KAFKA_LEADER_EPOCH;
+import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.KAFKA_MAX_OFFSET;
+import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.KAFKA_OFFSET;
+import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.KAFKA_PARTITION;
+import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.KAFKA_TOPIC;
+
+public class PublishKafkaUtil {
+
+    /**
+     * Polls for a batch of FlowFiles that should be published to Kafka within a single transaction
+     * @param session the process session to poll from
+     * @return the FlowFiles that should be sent as a single transaction
+     */
+    public static List<FlowFile> pollFlowFiles(final ProcessSession session) {
+        final List<FlowFile> initialFlowFiles = session.get(FlowFileFilters.newSizeBasedFilter(1, DataUnit.MB, 500));
+        if (initialFlowFiles.isEmpty()) {
+            return initialFlowFiles;
+        }
+
+        // Check if any of the FlowFiles indicate that the consumer offsets have yet to be committed.
+        boolean offsetsCommitted = true;
+        for (final FlowFile flowFile : initialFlowFiles) {
+            if ("false".equals(flowFile.getAttribute(KafkaProcessorUtils.KAFKA_CONSUMER_OFFSETS_COMMITTED))) {
+                offsetsCommitted = false;
+                break;
+            }
+        }
+
+        if (offsetsCommitted) {
+            return initialFlowFiles;
+        }
+
+        // If we need to commit consumer offsets, it is important that we retrieve all FlowFiles that may be available. Otherwise, we could
+        // have a situation in which there are 2 FlowFiles for Topic MyTopic and Partition 1. The first FlowFile may have an offset of 100,000
+        // while the second has an offset of 98,000. If we gather only the first, we could commit 100,000 offset before processing offset 98,000.
+        // To avoid that, we consume all FlowFiles in the queue. It's important also that all FlowFiles that have been consumed from Kafka are made
+        // available in the queue. This can be done by using a ProcessGroup with Batch Output, as described in the additionalDetails of the Kafka Processors.
+        return pollAllFlowFiles(session, initialFlowFiles);
+    }
+
+    private static List<FlowFile> pollAllFlowFiles(final ProcessSession session, final List<FlowFile> initialFlowFiles) {
+        final List<FlowFile> polled = new ArrayList<>(initialFlowFiles);
+        while (true) {
+            final List<FlowFile> flowFiles = session.get(10_000);
+            if (flowFiles.isEmpty()) {
+                break;
+            }
+
+            polled.addAll(flowFiles);
+        }
+
+        return polled;
+    }
+
+    /**
+     * Adds the appropriate Kafka Consumer offsets to the active transaction of the given publisher lease
+     * @param lease the lease that has an open transaction
+     * @param flowFile the FlowFile whose offsets should be acknowledged
+     * @param logger the processor's logger
+     */
+    public static void addConsumerOffsets(final PublisherLease lease, final FlowFile flowFile, final ComponentLog logger) {
+        final String topic = flowFile.getAttribute(KafkaProcessorUtils.KAFKA_TOPIC);
+        final Long partition = getNumericAttribute(flowFile, KafkaProcessorUtils.KAFKA_PARTITION, logger);
+        Long maxOffset = getNumericAttribute(flowFile, KafkaProcessorUtils.KAFKA_MAX_OFFSET, logger);
+        if (maxOffset == null) {
+            maxOffset = getNumericAttribute(flowFile, KafkaProcessorUtils.KAFKA_OFFSET, logger);
+        }
+
+        final Long epoch = getNumericAttribute(flowFile, KafkaProcessorUtils.KAFKA_LEADER_EPOCH, logger);
+        final String consumerGroupId = flowFile.getAttribute(KAFKA_CONSUMER_GROUP_ID);
+
+        if (topic == null || partition == null || maxOffset == null || consumerGroupId == null) {
+            logger.warn("Cannot commit consumer offsets because at least one of the following FlowFile attributes is missing from {}: {}", flowFile,
+                Arrays.asList(KAFKA_TOPIC, KAFKA_PARTITION, KAFKA_MAX_OFFSET + " (or " + KAFKA_OFFSET + ")", KAFKA_LEADER_EPOCH, KAFKA_CONSUMER_GROUP_ID));
+            return;
+        }
+
+        lease.ackConsumerOffsets(topic, partition.intValue(), maxOffset, epoch == null ? null : epoch.intValue(), consumerGroupId);
+    }
+
+    private static Long getNumericAttribute(final FlowFile flowFile, final String attributeName, final ComponentLog logger) {
+        final String attributeValue = flowFile.getAttribute(attributeName);
+        if (attributeValue == null) {
+            return null;
+        }
+
+        try {
+            return Long.parseLong(attributeValue);
+        } catch (final NumberFormatException nfe) {
+            logger.warn("Expected a numeric value for attribute '{}' but found non-numeric value for {}", attributeName, flowFile);
+            return null;
+        }
+    }
+}
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafka_2_6.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafka_2_6.java
index 9f6bdc7..e9f5930 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafka_2_6.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafka_2_6.java
@@ -29,6 +29,7 @@ import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.annotation.lifecycle.OnStopped;
 import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.ConfigVerificationResult;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
@@ -36,7 +37,6 @@ import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.AbstractProcessor;
-import org.apache.nifi.components.ConfigVerificationResult;
 import org.apache.nifi.processor.DataUnit;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSession;
@@ -44,7 +44,6 @@ import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.processor.VerifiableProcessor;
 import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.io.InputStreamCallback;
-import org.apache.nifi.processor.util.FlowFileFilters;
 import org.apache.nifi.processor.util.StandardValidators;
 
 import javax.xml.bind.DatatypeConverter;
@@ -252,6 +251,7 @@ public class PublishKafka_2_6 extends AbstractProcessor implements VerifiablePro
         .description("When Use Transaction is set to true, KafkaProducer config 'transactional.id' will be a generated UUID and will be prefixed with this string.")
         .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
         .addValidator(StandardValidators.NON_EMPTY_EL_VALIDATOR)
+        .dependsOn(USE_TRANSACTIONS, "true")
         .required(false)
         .build();
     static final PropertyDescriptor MESSAGE_HEADER_ENCODING = new PropertyDescriptor.Builder()
@@ -281,17 +281,28 @@ public class PublishKafka_2_6 extends AbstractProcessor implements VerifiablePro
 
     static {
         final List<PropertyDescriptor> properties = new ArrayList<>();
-        properties.addAll(KafkaProcessorUtils.getCommonPropertyDescriptors());
+        properties.add(KafkaProcessorUtils.BOOTSTRAP_SERVERS);
         properties.add(TOPIC);
-        properties.add(DELIVERY_GUARANTEE);
-        properties.add(FAILURE_STRATEGY);
         properties.add(USE_TRANSACTIONS);
         properties.add(TRANSACTIONAL_ID_PREFIX);
+        properties.add(MESSAGE_DEMARCATOR);
+        properties.add(KafkaProcessorUtils.FAILURE_STRATEGY);
+        properties.add(DELIVERY_GUARANTEE);
         properties.add(ATTRIBUTE_NAME_REGEX);
         properties.add(MESSAGE_HEADER_ENCODING);
+        properties.add(KafkaProcessorUtils.SECURITY_PROTOCOL);
+        properties.add(KafkaProcessorUtils.SASL_MECHANISM);
+        properties.add(KafkaProcessorUtils.KERBEROS_CREDENTIALS_SERVICE);
+        properties.add(KafkaProcessorUtils.SELF_CONTAINED_KERBEROS_USER_SERVICE);
+        properties.add(KafkaProcessorUtils.JAAS_SERVICE_NAME);
+        properties.add(KafkaProcessorUtils.USER_PRINCIPAL);
+        properties.add(KafkaProcessorUtils.USER_KEYTAB);
+        properties.add(KafkaProcessorUtils.USERNAME);
+        properties.add(KafkaProcessorUtils.PASSWORD);
+        properties.add(KafkaProcessorUtils.TOKEN_AUTH);
+        properties.add(KafkaProcessorUtils.SSL_CONTEXT_SERVICE);
         properties.add(KEY);
         properties.add(KEY_ATTRIBUTE_ENCODING);
-        properties.add(MESSAGE_DEMARCATOR);
         properties.add(MAX_REQUEST_SIZE);
         properties.add(ACK_WAIT_TIME);
         properties.add(METADATA_WAIT_TIME);
@@ -405,7 +416,7 @@ public class PublishKafka_2_6 extends AbstractProcessor implements VerifiablePro
     public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
         final boolean useDemarcator = context.getProperty(MESSAGE_DEMARCATOR).isSet();
 
-        final List<FlowFile> flowFiles = session.get(FlowFileFilters.newSizeBasedFilter(250, DataUnit.KB, 500));
+        final List<FlowFile> flowFiles = PublishKafkaUtil.pollFlowFiles(session);
         if (flowFiles.isEmpty()) {
             return;
         }
@@ -460,6 +471,11 @@ public class PublishKafka_2_6 extends AbstractProcessor implements VerifiablePro
                             }
                         }
                     });
+
+                    // If consumer offsets haven't been committed, add them to the transaction.
+                    if (useTransactions && "false".equals(flowFile.getAttribute(KafkaProcessorUtils.KAFKA_CONSUMER_OFFSETS_COMMITTED))) {
+                        PublishKafkaUtil.addConsumerOffsets(lease, flowFile, getLogger());
+                    }
                 }
 
                 // Complete the send
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublisherLease.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublisherLease.java
index f66b8b1..e87c560 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublisherLease.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublisherLease.java
@@ -17,16 +17,20 @@
 
 package org.apache.nifi.processors.kafka.pubsub;
 
+import org.apache.kafka.clients.consumer.OffsetAndMetadata;
 import org.apache.kafka.clients.producer.Callback;
 import org.apache.kafka.clients.producer.Producer;
 import org.apache.kafka.clients.producer.ProducerRecord;
 import org.apache.kafka.clients.producer.RecordMetadata;
 import org.apache.kafka.common.PartitionInfo;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.errors.FencedInstanceIdException;
+import org.apache.kafka.common.errors.ProducerFencedException;
 import org.apache.kafka.common.header.Headers;
-import org.apache.nifi.flowfile.FlowFile;
-import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.components.ConfigVerificationResult;
 import org.apache.nifi.components.ConfigVerificationResult.Outcome;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.schema.access.SchemaNotFoundException;
 import org.apache.nifi.serialization.RecordSetWriter;
 import org.apache.nifi.serialization.RecordSetWriterFactory;
@@ -48,6 +52,7 @@ import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicLong;
@@ -262,6 +267,14 @@ public class PublisherLease implements Closeable {
         tracker.incrementSentCount(flowFile);
     }
 
+    void ackConsumerOffsets(final String topic, final int partition, final long offset, final Integer leaderEpoch, final String consumerGroupId) {
+        final TopicPartition topicPartition = new TopicPartition(topic, partition);
+        final OffsetAndMetadata offsetAndMetadata = new OffsetAndMetadata(offset + 1, Optional.ofNullable(leaderEpoch), null);
+        final Map<TopicPartition, OffsetAndMetadata> offsetMap = Collections.singletonMap(topicPartition, offsetAndMetadata);
+
+        logger.debug("Acknowledging Consumer Offsets for topic={}, partition={}, offset={}, consumerGroup={}, leaderEpoch={}", topic, partition, offset, consumerGroupId, leaderEpoch);
+        producer.sendOffsetsToTransaction(offsetMap, consumerGroupId);
+    }
 
     public PublishResult complete() {
         if (tracker == null) {
@@ -280,6 +293,8 @@ public class PublisherLease implements Closeable {
                 producer.commitTransaction();
                 activeTransaction = false;
             }
+        } catch (final ProducerFencedException | FencedInstanceIdException e) {
+            throw e;
         } catch (final Exception e) {
             poison();
             throw e;
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublisherPool.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublisherPool.java
index 9e2c30f..bd3a664 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublisherPool.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublisherPool.java
@@ -80,9 +80,24 @@ public class PublisherPool implements Closeable {
 
         final Producer<byte[], byte[]> producer = new KafkaProducer<>(properties);
         final PublisherLease lease = new PublisherLease(producer, maxMessageSize, maxAckWaitMillis, logger, useTransactions, attributeNameRegex, headerCharacterSet) {
+            private volatile boolean closed = false;
+
             @Override
             public void close() {
+                if (isPoisoned() && useTransactions && !closed) {
+                    try {
+                        producer.abortTransaction();
+                    } catch (final Exception e) {
+                        logger.error("Failed to abort producer transaction", e);
+                    }
+                }
+
                 if (isPoisoned() || isClosed()) {
+                    if (closed) {
+                        return;
+                    }
+
+                    closed = true;
                     super.close();
                 } else {
                     publisherQueue.offer(this);
diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/ConsumerPoolTest.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/ConsumerPoolTest.java
index 24d94bc..bea87bd 100644
--- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/ConsumerPoolTest.java
+++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/ConsumerPoolTest.java
@@ -17,6 +17,7 @@
 package org.apache.nifi.processors.kafka.pubsub;
 
 import org.apache.kafka.clients.consumer.Consumer;
+import org.apache.kafka.clients.consumer.ConsumerGroupMetadata;
 import org.apache.kafka.clients.consumer.ConsumerRecord;
 import org.apache.kafka.clients.consumer.ConsumerRecords;
 import org.apache.kafka.common.KafkaException;
@@ -83,7 +84,8 @@ public class ConsumerPoolTest {
                 true,
                 StandardCharsets.UTF_8,
                 null,
-                null) {
+                null,
+                true) {
             @Override
             protected Consumer<byte[], byte[]> createKafkaConsumer() {
                 return consumer;
@@ -103,7 +105,8 @@ public class ConsumerPoolTest {
                 true,
                 StandardCharsets.UTF_8,
                 Pattern.compile(".*"),
-                null) {
+                null,
+                true) {
             @Override
             protected Consumer<byte[], byte[]> createKafkaConsumer() {
                 return consumer;
@@ -147,13 +150,14 @@ public class ConsumerPoolTest {
         final ConsumerRecords<byte[], byte[]> firstRecs = createConsumerRecords("foo", 1, 1L, firstPassValues);
 
         when(consumer.poll(any(Duration.class))).thenReturn(firstRecs, createConsumerRecords("nifi", 0, 0L, new byte[][]{}));
+        when(consumer.groupMetadata()).thenReturn(mock(ConsumerGroupMetadata.class));
         try (final ConsumerLease lease = testPool.obtainConsumer(mockSession, mockContext)) {
             lease.poll();
             lease.commit();
         }
         testPool.close();
         verify(mockSession, times(3)).create();
-        verify(mockSession, times(1)).commitAsync(Mockito.any(Runnable.class));
+        verify(mockSession, times(1)).commitAsync(Mockito.any(Runnable.class), Mockito.any(java.util.function.Consumer.class));
         final PoolStats stats = testPool.getPoolStats();
         assertEquals(1, stats.consumerCreatedCount);
         assertEquals(1, stats.consumerClosedCount);
@@ -192,7 +196,8 @@ public class ConsumerPoolTest {
             true,
             StandardCharsets.UTF_8,
             null,
-            new int[] {1, 2, 3}) {
+            new int[] {1, 2, 3},
+            true) {
             @Override
             protected Consumer<byte[], byte[]> createKafkaConsumer() {
                 return consumer;
@@ -272,13 +277,14 @@ public class ConsumerPoolTest {
         final ConsumerRecords<byte[], byte[]> firstRecs = createConsumerRecords("foo", 1, 1L, firstPassValues);
 
         when(consumer.poll(any(Duration.class))).thenReturn(firstRecs, createConsumerRecords("nifi", 0, 0L, new byte[][]{}));
+        when(consumer.groupMetadata()).thenReturn(mock(ConsumerGroupMetadata.class));
         try (final ConsumerLease lease = testDemarcatedPool.obtainConsumer(mockSession, mockContext)) {
             lease.poll();
             lease.commit();
         }
         testDemarcatedPool.close();
         verify(mockSession, times(1)).create();
-        verify(mockSession, times(1)).commitAsync(Mockito.any(Runnable.class));
+        verify(mockSession, times(1)).commitAsync(Mockito.any(Runnable.class), Mockito.any(java.util.function.Consumer.class));
         final PoolStats stats = testDemarcatedPool.getPoolStats();
         assertEquals(1, stats.consumerCreatedCount);
         assertEquals(1, stats.consumerClosedCount);
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestPutEmail.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestPutEmail.java
index 311869f..7c18570 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestPutEmail.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestPutEmail.java
@@ -16,34 +16,33 @@
  */
 package org.apache.nifi.processors.standard;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-
-import java.io.InputStream;
-import java.nio.charset.StandardCharsets;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
 import jakarta.mail.BodyPart;
 import jakarta.mail.Message;
 import jakarta.mail.MessagingException;
 import jakarta.mail.internet.MimeMessage.RecipientType;
 import jakarta.mail.internet.MimeMultipart;
 import jakarta.mail.internet.MimeUtility;
-
 import org.apache.commons.codec.binary.Base64;
 import org.apache.commons.codec.binary.StringUtils;
 import org.apache.commons.io.IOUtils;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
-import org.apache.nifi.util.LogMessage;
 import org.apache.nifi.util.TestRunner;
 import org.apache.nifi.util.TestRunners;
 import org.junit.Before;
 import org.junit.Test;
 
+import java.io.InputStream;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
 public class TestPutEmail {
 
     /**
@@ -215,8 +214,7 @@ public class TestPutEmail {
         runner.assertAllFlowFilesTransferred(PutEmail.REL_FAILURE);
 
         assertEquals("Expected no messages to be sent", 0, processor.getMessages().size());
-        final LogMessage logMessage = runner.getLogger().getErrorMessages().get(0);
-        assertTrue(((String)logMessage.getArgs()[2]).contains("Required property 'From' evaluates to an empty string"));
+        assertFalse(runner.getLogger().getErrorMessages().isEmpty());
     }
 
     @Test
diff --git a/nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor-nar/pom.xml b/nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor-nar/pom.xml
new file mode 100644
index 0000000..c909d4a
--- /dev/null
+++ b/nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor-nar/pom.xml
@@ -0,0 +1,43 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements. See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License. You may obtain a copy of the License at
+  http://www.apache.org/licenses/LICENSE-2.0
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <artifactId>nifi-stateless-processor-bundle</artifactId>
+        <groupId>org.apache.nifi</groupId>
+        <version>1.15.0-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+
+    <artifactId>nifi-stateless-processor-nar</artifactId>
+    <packaging>nar</packaging>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-stateless-processor</artifactId>
+            <version>1.15.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-standard-services-api-nar</artifactId>
+            <version>1.15.0-SNAPSHOT</version>
+            <type>nar</type>
+        </dependency>
+    </dependencies>
+
+</project>
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor-tests/pom.xml b/nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor-tests/pom.xml
new file mode 100644
index 0000000..2e71ce9
--- /dev/null
+++ b/nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor-tests/pom.xml
@@ -0,0 +1,170 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements. See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License. You may obtain a copy of the License at
+  http://www.apache.org/licenses/LICENSE-2.0
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <artifactId>nifi-stateless-processor-bundle</artifactId>
+        <groupId>org.apache.nifi</groupId>
+        <version>1.15.0-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+
+    <artifactId>nifi-stateless-processor-tests</artifactId>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+            <version>1.15.0-SNAPSHOT</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-framework-api</artifactId>
+            <version>1.15.0-SNAPSHOT</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-server-api</artifactId>
+            <version>1.15.0-SNAPSHOT</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-runtime</artifactId>
+            <version>1.15.0-SNAPSHOT</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-nar-utils</artifactId>
+            <version>1.15.0-SNAPSHOT</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-stateless-processor</artifactId>
+            <version>1.15.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-ssl-context-service-api</artifactId>
+            <version>1.15.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi.registry</groupId>
+            <artifactId>nifi-registry-client</artifactId>
+            <version>1.15.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi.registry</groupId>
+            <artifactId>nifi-registry-data-model</artifactId>
+            <version>1.15.0-SNAPSHOT</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-mock</artifactId>
+            <version>1.15.0-SNAPSHOT</version>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-stateless-nar</artifactId>
+            <version>1.15.0-SNAPSHOT</version>
+            <type>nar</type>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-update-attribute-nar</artifactId>
+            <version>1.15.0-SNAPSHOT</version>
+            <type>nar</type>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-standard-nar</artifactId>
+            <version>1.15.0-SNAPSHOT</version>
+            <type>nar</type>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-standard-services-api-nar</artifactId>
+            <version>1.15.0-SNAPSHOT</version>
+            <type>nar</type>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-jetty-bundle</artifactId>
+            <version>1.15.0-SNAPSHOT</version>
+            <type>nar</type>
+        </dependency>
+    </dependencies>
+
+    <build>
+        <plugins>
+            <plugin>
+                <artifactId>maven-assembly-plugin</artifactId>
+                <configuration>
+                    <finalName>nifi-stateless-processors-test-assembly</finalName>
+                    <attach>true</attach>
+                    <appendAssemblyId>false</appendAssemblyId>
+                </configuration>
+                <executions>
+                    <execution>
+                        <id>make shared resource</id>
+                        <goals>
+                            <goal>single</goal>
+                        </goals>
+                        <phase>process-test-resources</phase>
+                        <configuration>
+                            <archiverConfig>
+                                <defaultDirectoryMode>0775</defaultDirectoryMode>
+                                <directoryMode>0775</directoryMode>
+                                <fileMode>0664</fileMode>
+                            </archiverConfig>
+                            <descriptors>
+                                <descriptor>src/test/assembly/dependencies.xml</descriptor>
+                            </descriptors>
+                            <tarLongFileMode>posix</tarLongFileMode>
+                            <formats>
+                                <format>dir</format>
+                            </formats>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <groupId>org.apache.rat</groupId>
+                <artifactId>apache-rat-plugin</artifactId>
+                <configuration>
+                    <excludes combine.children="append">
+                        <exclude>src/test/resources/passthrough-flow.json</exclude>
+                        <exclude>src/test/resources/split-text.json</exclude>
+                        <exclude>src/test/resources/route-one-to-failure.json</exclude>
+                        <exclude>src/test/resources/route-to-desired-port.json</exclude>
+                        <exclude>src/test/resources/route-to-failure-inner-group.json</exclude>
+                        <exclude>src/test/resources/sleep.json</exclude>
+                        <exclude>src/test/resources/throw-exception.json</exclude>
+                        <exclude>src/test/resources/log-message.json</exclude>
+                    </excludes>
+                </configuration>
+            </plugin>
+        </plugins>
+    </build>
+</project>
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor-tests/src/test/assembly/dependencies.xml b/nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor-tests/src/test/assembly/dependencies.xml
new file mode 100644
index 0000000..743db97
--- /dev/null
+++ b/nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor-tests/src/test/assembly/dependencies.xml
@@ -0,0 +1,62 @@
+<?xml version="1.0"?>
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+      http://www.apache.org/licenses/LICENSE-2.0
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+<assembly>
+    <id>bin</id>
+    <includeBaseDirectory>false</includeBaseDirectory>
+    <baseDirectory>nifi-execute-stateless-processor-assembly-${project.version}</baseDirectory>
+
+    <dependencySets>
+        <!-- Write out all dependency artifacts to directory -->
+        <dependencySet>
+            <scope>runtime</scope>
+            <useProjectArtifact>false</useProjectArtifact>
+            <outputDirectory>.</outputDirectory>
+            <directoryMode>0770</directoryMode>
+            <fileMode>0664</fileMode>
+            <useTransitiveFiltering>true</useTransitiveFiltering>
+            <includes>
+                <include>nifi-stateless-bootstrap</include>
+                <include>nifi-stateless-nar</include>
+                <include>nifi-standard-nar</include>
+                <include>nifi-standard-services-api-nar</include>
+                <include>nifi-jetty-bundle</include>
+                <include>nifi-update-attribute-nar</include>
+                <include>nifi-api</include>
+                <include>nifi-framework-api</include>
+                <include>nifi-server-api</include>
+                <include>nifi-runtime</include>
+                <include>nifi-nar-utils</include>
+            </includes>
+        </dependencySet>
+
+        <!-- Write out the bootstrap libs for java11 to its own dir -->
+        <!-- TODO: remove this dependency set once minimum Java version is 11 -->
+        <dependencySet>
+            <scope>runtime</scope>
+            <useProjectArtifact>false</useProjectArtifact>
+            <outputDirectory>./java11</outputDirectory>
+            <directoryMode>0770</directoryMode>
+            <fileMode>0664</fileMode>
+            <useTransitiveFiltering>true</useTransitiveFiltering>
+            <includes>
+                <include>org.glassfish.jaxb:jaxb-runtime</include>
+                <include>jakarta.xml.bind:jakarta.xml.bind-api</include>
+                <include>javax.annotation:javax.annotation-api</include>
+            </includes>
+        </dependencySet>
+    </dependencySets>
+
+</assembly>
diff --git a/nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor-tests/src/test/java/org/apache/nifi/processors/stateless/TestExecuteStateless.java b/nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor-tests/src/test/java/org/apache/nifi/processors/stateless/TestExecuteStateless.java
new file mode 100644
index 0000000..b1423bd
--- /dev/null
+++ b/nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor-tests/src/test/java/org/apache/nifi/processors/stateless/TestExecuteStateless.java
@@ -0,0 +1,207 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.processors.stateless;
+
+import org.apache.nifi.util.LogMessage;
+import org.apache.nifi.util.MockComponentLog;
+import org.apache.nifi.util.MockFlowFile;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.function.Function;
+
+import static junit.framework.TestCase.assertTrue;
+import static org.junit.Assert.assertEquals;
+
+public class TestExecuteStateless {
+    private static final String HELLO_WORLD = "Hello World";
+    private static final String LIB_DIR = "target/nifi-stateless-processors-test-assembly";
+    private static final String WORK_DIR = "target/work";
+
+    private TestRunner runner;
+
+    @Before
+    public void setup() {
+        runner = TestRunners.newTestRunner(ExecuteStateless.class);
+        runner.setProperty(ExecuteStateless.DATAFLOW_SPECIFICATION_STRATEGY, ExecuteStateless.SPEC_FROM_FILE);
+        runner.setProperty(ExecuteStateless.LIB_DIRECTORY, LIB_DIR);
+        runner.setProperty(ExecuteStateless.WORKING_DIRECTORY, WORK_DIR);
+    }
+
+    @Test
+    public void testSimplePassThrough() {
+        runner.setProperty(ExecuteStateless.DATAFLOW_FILE, "src/test/resources/passthrough-flow.json");
+        runner.setProperty(ExecuteStateless.INPUT_PORT, "In");
+
+        runner.enqueue(HELLO_WORLD.getBytes(), Collections.singletonMap("abc", "xyz"));
+        runner.run();
+
+        runner.assertTransferCount(ExecuteStateless.REL_OUTPUT, 1);
+        final MockFlowFile output = runner.getFlowFilesForRelationship(ExecuteStateless.REL_OUTPUT).get(0);
+        output.assertAttributeEquals("abc", "xyz");
+        output.assertContentEquals(HELLO_WORLD);
+    }
+
+    @Test
+    public void testSplitWithParameters() {
+        runner.setProperty(ExecuteStateless.DATAFLOW_FILE, "src/test/resources/split-text.json");
+        runner.setProperty(ExecuteStateless.INPUT_PORT, "In");
+        runner.setProperty("Lines Per Split", "3");
+
+        runner.enqueue("The\nQuick\nBrown\nFox\nJumps\nOver\nThe\nLazy\nDog".getBytes(), Collections.singletonMap("abc", "xyz"));
+        runner.run();
+
+        runner.assertTransferCount(ExecuteStateless.REL_OUTPUT, 3);
+        final List<MockFlowFile> output = runner.getFlowFilesForRelationship(ExecuteStateless.REL_OUTPUT);
+        output.forEach(ff -> ff.assertAttributeEquals("abc", "xyz"));
+        output.get(0).assertContentEquals("The\nQuick\nBrown");
+        output.get(1).assertContentEquals("Fox\nJumps\nOver");
+        output.get(2).assertContentEquals("The\nLazy\nDog");
+    }
+
+    @Test
+    public void testRouteToFailure() {
+        runner.setProperty(ExecuteStateless.DATAFLOW_FILE, "src/test/resources/route-one-to-failure.json");
+        runner.setProperty(ExecuteStateless.FAILURE_PORTS, "Last");
+
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ExecuteStateless.REL_OUTPUT, 0);
+    }
+
+
+    @Test
+    public void testRouteToFailureWithInput() {
+        runner.setProperty(ExecuteStateless.DATAFLOW_FILE, "src/test/resources/route-to-desired-port.json");
+        runner.setProperty(ExecuteStateless.INPUT_PORT, "In");
+        runner.setProperty(ExecuteStateless.FAILURE_PORTS, "Other");
+
+        runner.enqueue("A", Collections.singletonMap("desired.port", "A"));
+        runner.enqueue("B", Collections.singletonMap("desired.port", "B"));
+        runner.enqueue("C", Collections.singletonMap("desired.port", "C"));
+
+        runner.run(3);
+
+        runner.assertTransferCount(ExecuteStateless.REL_OUTPUT, 3);
+        runner.assertTransferCount(ExecuteStateless.REL_ORIGINAL, 3);
+        runner.assertTransferCount(ExecuteStateless.REL_FAILURE, 0);
+        runner.assertTransferCount(ExecuteStateless.REL_TIMEOUT, 0);
+
+        runner.clearTransferState();
+        runner.enqueue("D", Collections.singletonMap("desired.port", "D"));
+        runner.run();
+        runner.assertAllFlowFilesTransferred(ExecuteStateless.REL_FAILURE, 1);
+        runner.getFlowFilesForRelationship(ExecuteStateless.REL_FAILURE).get(0).assertAttributeEquals("failure.port.name", "Other");
+    }
+
+
+    @Test
+    public void testMultipleFailurePortNames() {
+        runner.setProperty(ExecuteStateless.DATAFLOW_FILE, "src/test/resources/route-to-desired-port.json");
+        runner.setProperty(ExecuteStateless.INPUT_PORT, "In");
+        runner.setProperty(ExecuteStateless.FAILURE_PORTS, "Other, A,    B,C");
+
+        runner.enqueue("B", Collections.singletonMap("desired.port", "B"));
+
+        runner.run();
+
+        runner.assertTransferCount(ExecuteStateless.REL_FAILURE, 1);
+        runner.getFlowFilesForRelationship(ExecuteStateless.REL_FAILURE).get(0).assertAttributeEquals("failure.port.name", "B");
+    }
+
+    @Test
+    public void testRouteToFailureInnerGroup() {
+        runner.setProperty(ExecuteStateless.DATAFLOW_FILE, "src/test/resources/route-to-failure-inner-group.json");
+        runner.setProperty(ExecuteStateless.INPUT_PORT, "In");
+        runner.setProperty(ExecuteStateless.FAILURE_PORTS, "failure");
+
+        runner.enqueue("Hello World");
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ExecuteStateless.REL_FAILURE, 1);
+        runner.getFlowFilesForRelationship(ExecuteStateless.REL_FAILURE).get(0).assertAttributeEquals("failure.port.name", "failure");
+    }
+
+    @Test
+    public void testTimeout() {
+        runner.setProperty(ExecuteStateless.DATAFLOW_FILE, "src/test/resources/sleep.json");
+        runner.setProperty(ExecuteStateless.INPUT_PORT, "In");
+        runner.setProperty(ExecuteStateless.DATAFLOW_TIMEOUT, "100 millis");
+        runner.setProperty("Duration", "5 sec"); // Have DebugFlow sleep for 5 seconds
+
+
+        runner.enqueue("Hello World");
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ExecuteStateless.REL_TIMEOUT, 1);
+    }
+
+    @Test
+    public void testProcessorExceptionRoutesToFailure() {
+        runner.setProperty(ExecuteStateless.DATAFLOW_FILE, "src/test/resources/throw-exception.json");
+        runner.setProperty(ExecuteStateless.INPUT_PORT, "In");
+
+        runner.enqueue("Hello World");
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ExecuteStateless.REL_FAILURE, 1);
+    }
+
+    @Test
+    public void testInfoBulletinNotSurfaced() {
+        testBulletinSurfaced("INFO", false, MockComponentLog::getInfoMessages);
+    }
+
+    @Test
+    public void testWarnBulletinSurfaced() {
+        testBulletinSurfaced("WARN", true, MockComponentLog::getWarnMessages);
+    }
+
+    @Test
+    public void testErrorBulletinSurfaced() {
+        testBulletinSurfaced("ERROR", true, MockComponentLog::getErrorMessages);
+    }
+
+    private void testBulletinSurfaced(final String logLevel, final boolean shouldBeSurfaced, final Function<MockComponentLog, List<LogMessage>> getMessageFunction) {
+        final String logMessage = "Unit Test Message";
+
+        runner.setProperty(ExecuteStateless.DATAFLOW_FILE, "src/test/resources/log-message.json");
+        runner.setProperty(ExecuteStateless.INPUT_PORT, "In");
+        runner.setProperty("Log Message", logMessage);
+        runner.setProperty("Log Level", logLevel);
+
+        runner.enqueue("Hello World");
+        runner.run();
+
+        runner.assertTransferCount(ExecuteStateless.REL_ORIGINAL, 1);
+        final List<LogMessage> logMessages = getMessageFunction.apply(runner.getLogger());
+        final long matchingMessageCount = logMessages.stream()
+            .filter(msg -> msg.getMsg().contains(logMessage))
+            .count();
+
+        if (shouldBeSurfaced) {
+            assertTrue(matchingMessageCount > 0);
+        } else {
+            assertEquals(0, matchingMessageCount);
+        }
+    }
+}
diff --git a/nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor-tests/src/test/resources/log-message.json b/nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor-tests/src/test/resources/log-message.json
new file mode 100644
index 0000000..b5bd51f
--- /dev/null
+++ b/nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor-tests/src/test/resources/log-message.json
@@ -0,0 +1 @@
+{"flowContents":{"identifier":"7bd4dd54-be68-3796-b607-fef79510b5cc","name":"log-message","comments":"","position":{"x":-3276.819356871901,"y":890.7631521990795},"processGroups":[],"remoteProcessGroups":[],"processors":[{"identifier":"c4fa5dd4-ce26-3878-bb92-8c91343c15e3","name":"LogMessage","comments":"","position":{"x":896.0,"y":368.0},"bundle":{"group":"org.apache.nifi","artifact":"nifi-standard-nar","version":"1.15.0-SNAPSHOT"},"style":{},"type":"org.apache.nifi.processors.standard.L [...]
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor-tests/src/test/resources/passthrough-flow.json b/nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor-tests/src/test/resources/passthrough-flow.json
new file mode 100644
index 0000000..a525a2c
--- /dev/null
+++ b/nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor-tests/src/test/resources/passthrough-flow.json
@@ -0,0 +1,90 @@
+{
+  "flowContents": {
+    "identifier": "f73ba15e-7e77-3fe0-adba-6afaba338e07",
+    "name": "Pass-Through",
+    "comments": "",
+    "position": {
+      "x": -953.5297283891521,
+      "y": 184.44227749043935
+    },
+    "processGroups": [],
+    "remoteProcessGroups": [],
+    "processors": [],
+    "inputPorts": [
+      {
+        "identifier": "63133e51-5219-335c-b3ea-56f8274fcbf0",
+        "name": "In",
+        "position": {
+          "x": 980.999958631427,
+          "y": 211.0000023873564
+        },
+        "type": "INPUT_PORT",
+        "concurrentlySchedulableTaskCount": 1,
+        "allowRemoteAccess": false,
+        "componentType": "INPUT_PORT",
+        "groupIdentifier": "f73ba15e-7e77-3fe0-adba-6afaba338e07"
+      }
+    ],
+    "outputPorts": [
+      {
+        "identifier": "42681445-23ba-3092-baca-6b206fb0298c",
+        "name": "Out",
+        "position": {
+          "x": 980.999958631427,
+          "y": 423.0000023873564
+        },
+        "type": "OUTPUT_PORT",
+        "concurrentlySchedulableTaskCount": 1,
+        "allowRemoteAccess": false,
+        "componentType": "OUTPUT_PORT",
+        "groupIdentifier": "f73ba15e-7e77-3fe0-adba-6afaba338e07"
+      }
+    ],
+    "connections": [
+      {
+        "identifier": "79f7be28-20c6-3d40-86a9-39b64132a68f",
+        "name": "",
+        "source": {
+          "id": "63133e51-5219-335c-b3ea-56f8274fcbf0",
+          "type": "INPUT_PORT",
+          "groupId": "f73ba15e-7e77-3fe0-adba-6afaba338e07",
+          "name": "In"
+        },
+        "destination": {
+          "id": "42681445-23ba-3092-baca-6b206fb0298c",
+          "type": "OUTPUT_PORT",
+          "groupId": "f73ba15e-7e77-3fe0-adba-6afaba338e07",
+          "name": "Out"
+        },
+        "labelIndex": 1,
+        "zIndex": 0,
+        "selectedRelationships": [
+          ""
+        ],
+        "backPressureObjectThreshold": 10000,
+        "backPressureDataSizeThreshold": "1 GB",
+        "flowFileExpiration": "0 sec",
+        "prioritizers": [],
+        "bends": [],
+        "loadBalanceStrategy": "DO_NOT_LOAD_BALANCE",
+        "partitioningAttribute": "",
+        "loadBalanceCompression": "DO_NOT_COMPRESS",
+        "componentType": "CONNECTION",
+        "groupIdentifier": "f73ba15e-7e77-3fe0-adba-6afaba338e07"
+      }
+    ],
+    "labels": [],
+    "funnels": [],
+    "controllerServices": [],
+    "variables": {},
+    "defaultFlowFileExpiration": "0 sec",
+    "defaultBackPressureObjectThreshold": 10000,
+    "defaultBackPressureDataSizeThreshold": "1 GB",
+    "flowFileConcurrency": "UNBOUNDED",
+    "flowFileOutboundPolicy": "STREAM_WHEN_AVAILABLE",
+    "componentType": "PROCESS_GROUP"
+  },
+  "externalControllerServices": {},
+  "parameterContexts": {},
+  "flowEncodingVersion": "1.0"
+}
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor-tests/src/test/resources/route-one-to-failure.json b/nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor-tests/src/test/resources/route-one-to-failure.json
new file mode 100644
index 0000000..bec9451
--- /dev/null
+++ b/nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor-tests/src/test/resources/route-one-to-failure.json
@@ -0,0 +1 @@
+{"flowContents":{"identifier":"07407a58-33d1-3f97-b29d-2a68ce0118f5","name":"Route One to Failure","comments":"","position":{"x":-1121.4711030366489,"y":525.0032298890848},"processGroups":[],"remoteProcessGroups":[],"processors":[{"identifier":"ce1cbd81-7d39-3b5d-bb58-1e30a39b3d91","name":"RouteOnAttribute","comments":"","position":{"x":836.9999593270581,"y":330.0000021146835},"bundle":{"group":"org.apache.nifi","artifact":"nifi-standard-nar","version":"1.15.0-SNAPSHOT"},"style":{},"type [...]
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor-tests/src/test/resources/route-to-desired-port.json b/nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor-tests/src/test/resources/route-to-desired-port.json
new file mode 100644
index 0000000..ed8262e
--- /dev/null
+++ b/nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor-tests/src/test/resources/route-to-desired-port.json
@@ -0,0 +1 @@
+{"flowContents":{"identifier":"9219030c-e73b-3cd5-8d99-b831dcd774be","name":"Route to Desired Port","comments":"","position":{"x":-1805.1150914043885,"y":541.6852281794169},"processGroups":[],"remoteProcessGroups":[],"processors":[{"identifier":"ef345c88-589c-3937-86fb-cc1373a47810","name":"RouteOnAttribute","comments":"","position":{"x":816.0,"y":288.0},"bundle":{"group":"org.apache.nifi","artifact":"nifi-standard-nar","version":"1.15.0-SNAPSHOT"},"style":{},"type":"org.apache.nifi.proc [...]
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor-tests/src/test/resources/route-to-failure-inner-group.json b/nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor-tests/src/test/resources/route-to-failure-inner-group.json
new file mode 100644
index 0000000..193fafb
--- /dev/null
+++ b/nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor-tests/src/test/resources/route-to-failure-inner-group.json
@@ -0,0 +1 @@
+{"flowContents":{"identifier":"eab15588-8b13-3f5b-bea0-980729c38bcb","name":"route-to-failure-inner-group","comments":"","position":{"x":-2573.4996342169,"y":562.3407266421189},"processGroups":[{"identifier":"e6cc89ab-a1d3-3c0d-9219-4f05038a7b8b","name":"inner","comments":"","position":{"x":1040.9999831843675,"y":240.99999718920503},"processGroups":[],"remoteProcessGroups":[],"processors":[{"identifier":"f717a940-48fb-33a5-ad85-b459a60f27cc","name":"DuplicateFlowFile","comments":"","posi [...]
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor-tests/src/test/resources/sleep.json b/nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor-tests/src/test/resources/sleep.json
new file mode 100644
index 0000000..1f448bb
--- /dev/null
+++ b/nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor-tests/src/test/resources/sleep.json
@@ -0,0 +1 @@
+{"flowContents":{"identifier":"cb50a692-ef3a-36e2-9ecc-548bc3a51e6c","name":"sleep","comments":"","position":{"x":-3275.786581948766,"y":596.422299105577},"processGroups":[],"remoteProcessGroups":[],"processors":[{"identifier":"0e3515cf-44aa-3555-baf4-e5c0e40eb344","name":"DebugFlow","comments":"","position":{"x":822.9999831843675,"y":258.99999718920503},"bundle":{"group":"org.apache.nifi","artifact":"nifi-standard-nar","version":"1.15.0-SNAPSHOT"},"style":{},"type":"org.apache.nifi.proc [...]
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor-tests/src/test/resources/split-text.json b/nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor-tests/src/test/resources/split-text.json
new file mode 100644
index 0000000..7582f16
--- /dev/null
+++ b/nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor-tests/src/test/resources/split-text.json
@@ -0,0 +1 @@
+{"flowContents":{"identifier":"3e6eeb4a-ecea-31da-ad8a-2a54d22e6ca4","name":"Split Text","comments":"","position":{"x":-1632.0,"y":128.0},"processGroups":[],"remoteProcessGroups":[],"processors":[{"identifier":"4b59285a-caad-38a5-9d88-335efac4f94f","name":"SplitText","comments":"","position":{"x":865.999958631427,"y":243.0000023873564},"bundle":{"group":"org.apache.nifi","artifact":"nifi-standard-nar","version":"1.15.0-SNAPSHOT"},"style":{},"type":"org.apache.nifi.processors.standard.Spl [...]
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor-tests/src/test/resources/throw-exception.json b/nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor-tests/src/test/resources/throw-exception.json
new file mode 100644
index 0000000..570e8b3
--- /dev/null
+++ b/nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor-tests/src/test/resources/throw-exception.json
@@ -0,0 +1 @@
+{"flowContents":{"identifier":"d5861174-a14e-3b70-b90d-47b0890a5b86","name":"throw-exception","comments":"","position":{"x":-3277.8521317950363,"y":369.21181601585585},"processGroups":[],"remoteProcessGroups":[],"processors":[{"identifier":"5fe27e72-6b0c-389a-bf4a-f1c5dff36c9d","name":"DebugFlow","comments":"","position":{"x":899.9999831843675,"y":243.99999718920503},"bundle":{"group":"org.apache.nifi","artifact":"nifi-standard-nar","version":"1.15.0-SNAPSHOT"},"style":{},"type":"org.apa [...]
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/pom.xml b/nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/pom.xml
new file mode 100644
index 0000000..f36df0b
--- /dev/null
+++ b/nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/pom.xml
@@ -0,0 +1,71 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements. See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License. You may obtain a copy of the License at
+  http://www.apache.org/licenses/LICENSE-2.0
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <artifactId>nifi-stateless-processor-bundle</artifactId>
+        <groupId>org.apache.nifi</groupId>
+        <version>1.15.0-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+
+    <artifactId>nifi-stateless-processor</artifactId>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+            <version>1.15.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-stateless-api</artifactId>
+            <version>1.15.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-utils</artifactId>
+            <version>1.15.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-stateless-bootstrap</artifactId>
+            <version>1.15.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-ssl-context-service-api</artifactId>
+            <version>1.15.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi.registry</groupId>
+            <artifactId>nifi-registry-client</artifactId>
+            <version>1.15.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi.registry</groupId>
+            <artifactId>nifi-registry-data-model</artifactId>
+            <version>1.15.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>com.fasterxml.jackson.core</groupId>
+            <artifactId>jackson-databind</artifactId>
+            <version>${jackson.version}</version>
+        </dependency>
+    </dependencies>
+
+</project>
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/ExecuteStateless.java b/nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/ExecuteStateless.java
new file mode 100644
index 0000000..cc06f1d
--- /dev/null
+++ b/nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/ExecuteStateless.java
@@ -0,0 +1,1041 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.processors.stateless;
+
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.behavior.SystemResource;
+import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
+import org.apache.nifi.annotation.behavior.SystemResourceConsiderations;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.PropertyDescriptor.Builder;
+import org.apache.nifi.components.Validator;
+import org.apache.nifi.components.resource.ResourceCardinality;
+import org.apache.nifi.components.resource.ResourceReference;
+import org.apache.nifi.components.resource.ResourceType;
+import org.apache.nifi.flow.Bundle;
+import org.apache.nifi.flow.VersionedConnection;
+import org.apache.nifi.flow.VersionedControllerService;
+import org.apache.nifi.flow.VersionedLabel;
+import org.apache.nifi.flow.VersionedPort;
+import org.apache.nifi.flow.VersionedProcessGroup;
+import org.apache.nifi.flow.VersionedProcessor;
+import org.apache.nifi.flow.VersionedRemoteGroupPort;
+import org.apache.nifi.flow.VersionedRemoteProcessGroup;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.logging.LogLevel;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.DataUnit;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processors.stateless.retrieval.CachingDataflowProvider;
+import org.apache.nifi.processors.stateless.retrieval.DataflowProvider;
+import org.apache.nifi.processors.stateless.retrieval.FileSystemDataflowProvider;
+import org.apache.nifi.processors.stateless.retrieval.RegistryDataflowProvider;
+import org.apache.nifi.registry.bucket.Bucket;
+import org.apache.nifi.registry.flow.VersionedFlow;
+import org.apache.nifi.registry.flow.VersionedFlowSnapshot;
+import org.apache.nifi.reporting.Bulletin;
+import org.apache.nifi.reporting.BulletinQuery;
+import org.apache.nifi.reporting.BulletinRepository;
+import org.apache.nifi.search.SearchContext;
+import org.apache.nifi.search.SearchResult;
+import org.apache.nifi.search.Searchable;
+import org.apache.nifi.ssl.SSLContextService;
+import org.apache.nifi.stateless.bootstrap.StatelessBootstrap;
+import org.apache.nifi.stateless.config.ExtensionClientDefinition;
+import org.apache.nifi.stateless.config.ParameterContextDefinition;
+import org.apache.nifi.stateless.config.ParameterValueProviderDefinition;
+import org.apache.nifi.stateless.config.ReportingTaskDefinition;
+import org.apache.nifi.stateless.config.SslContextDefinition;
+import org.apache.nifi.stateless.config.StatelessConfigurationException;
+import org.apache.nifi.stateless.engine.StatelessEngineConfiguration;
+import org.apache.nifi.stateless.flow.DataflowDefinition;
+import org.apache.nifi.stateless.flow.DataflowTrigger;
+import org.apache.nifi.stateless.flow.DataflowTriggerContext;
+import org.apache.nifi.stateless.flow.FailurePortEncounteredException;
+import org.apache.nifi.stateless.flow.StatelessDataflow;
+import org.apache.nifi.stateless.flow.TransactionThresholds;
+import org.apache.nifi.stateless.flow.TriggerResult;
+import org.apache.nifi.stream.io.StreamUtils;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingDeque;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.apache.nifi.expression.ExpressionLanguageScope.FLOWFILE_ATTRIBUTES;
+import static org.apache.nifi.expression.ExpressionLanguageScope.NONE;
+import static org.apache.nifi.processor.util.StandardValidators.DATA_SIZE_VALIDATOR;
+import static org.apache.nifi.processor.util.StandardValidators.NON_EMPTY_VALIDATOR;
+import static org.apache.nifi.processor.util.StandardValidators.POSITIVE_INTEGER_VALIDATOR;
+import static org.apache.nifi.processor.util.StandardValidators.TIME_PERIOD_VALIDATOR;
+import static org.apache.nifi.processor.util.StandardValidators.URL_VALIDATOR;
+import static org.apache.nifi.processor.util.StandardValidators.createDirectoryExistsValidator;
+
+@Restricted
+@SupportsBatching
+@SystemResourceConsiderations({
+    @SystemResourceConsideration(resource= SystemResource.CPU),
+    @SystemResourceConsideration(resource= SystemResource.DISK),
+    @SystemResourceConsideration(resource= SystemResource.MEMORY),
+    @SystemResourceConsideration(resource= SystemResource.NETWORK)
+})
+@DynamicProperty(name="Any Parameter name", value="Any value", description = "Any dynamic property that is added will be provided to the stateless flow as a Parameter. The name of the property will" +
+    " be the name of the Parameter, and the value of the property will be the value of the Parameter. Because Parameter values may or may not be sensitive, all dynamic properties will be considered" +
+    " sensitive in order to protect their integrity.")
+@InputRequirement(Requirement.INPUT_ALLOWED)
+@CapabilityDescription("Runs the configured dataflow using the Stateless NiFi engine. Please see documentation in order to understand the differences between the traditional NiFi runtime engine and" +
+    " the Stateless NiFi engine. If the Processor is configured with an incoming connection, the incoming FlowFiles will be queued up into the specified Input Port in the dataflow. Data that is" +
+    " transferred out of the flow via an Output Port will be sent to the 'output' relationship, and an attribute will be added to indicate which Port that FlowFile was transferred to. See" +
+    " Additional Details for more information.")
+@WritesAttributes({
+    @WritesAttribute(attribute="output.port.name", description = "The name of the Output Port that the FlowFile was transferred to"),
+    @WritesAttribute(attribute="failure.port.name", description = "If one or more FlowFiles is routed to one of the Output Ports that is configured as a Failure Port, the input FlowFile (if any) " +
+        "will have this attribute added to it, indicating the name of the Port that caused the dataflow to be considered a failure.")
+})
+public class ExecuteStateless extends AbstractProcessor implements Searchable {
+    public static final AllowableValue SPEC_FROM_FILE = new AllowableValue("Use Local File", "Use Local File or URL",
+        "Dataflow to run is stored as a file on the NiFi server or at a URL that is accessible to the NiFi server");
+    public static final AllowableValue SPEC_FROM_REGISTRY = new AllowableValue("Use NiFi Registry", "Use NiFi Registry", "Dataflow to run is stored in NiFi Registry");
+
+    public static final AllowableValue CONTENT_STORAGE_HEAP = new AllowableValue("Store Content on Heap", "Store Content on Heap",
+        "The FlowFile content will be stored on the NiFi JVM's heap. This is the most " +
+        "efficient option for small FlowFiles but can quickly exhaust the heap with larger FlowFiles, resulting in Out Of Memory Errors and node instability.");
+    public static final AllowableValue CONTENT_STORAGE_DISK = new AllowableValue("Store Content on Disk", "Store Content on Disk",
+        "The FlowFile content will be stored on disk, within the configured Work Directory. The content will still be cleared between invocations and will not be persisted across restarts.");
+
+    public static final PropertyDescriptor DATAFLOW_SPECIFICATION_STRATEGY = new Builder()
+        .name("Dataflow Specification Strategy")
+        .displayName("Dataflow Specification Strategy")
+        .description("Specifies how the Processor should obtain a copy of the dataflow that it is to run")
+        .required(true)
+        .allowableValues(SPEC_FROM_FILE, SPEC_FROM_REGISTRY)
+        .defaultValue(SPEC_FROM_FILE.getValue())
+        .build();
+
+    public static final PropertyDescriptor DATAFLOW_FILE = new Builder()
+        .name("Dataflow File")
+        .displayName("Dataflow File/URL")
+        .description("The filename or URL that specifies the dataflow that is to be run")
+        .required(true)
+        .identifiesExternalResource(ResourceCardinality.SINGLE, ResourceType.FILE, ResourceType.URL)
+        .dependsOn(DATAFLOW_SPECIFICATION_STRATEGY, SPEC_FROM_FILE)
+        .build();
+
+    public static final PropertyDescriptor REGISTRY_URL = new Builder()
+        .name("Registry URL")
+        .displayName("Registry URL")
+        .description("The URL of the NiFi Registry to retrieve the flow from")
+        .required(true)
+        .addValidator(URL_VALIDATOR)
+        .dependsOn(DATAFLOW_SPECIFICATION_STRATEGY, SPEC_FROM_REGISTRY)
+        .build();
+
+    public static final PropertyDescriptor SSL_CONTEXT_SERVICE = new Builder()
+        .name("Registry SSL Context Service")
+        .displayName("Registry SSL Context Service")
+        .description("The SSL Context Service to use for interacting with the NiFi Registry")
+        .required(false)
+        .identifiesControllerService(SSLContextService.class)
+        .dependsOn(DATAFLOW_SPECIFICATION_STRATEGY, SPEC_FROM_REGISTRY)
+        .build();
+
+    public static final PropertyDescriptor COMMS_TIMEOUT = new Builder()
+        .name("Communications Timeout")
+        .displayName("Communications Timeout")
+        .description("Specifies how long to wait before timing out when attempting to communicate with NiFi Registry")
+        .required(true)
+        .addValidator(TIME_PERIOD_VALIDATOR)
+        .dependsOn(DATAFLOW_SPECIFICATION_STRATEGY, SPEC_FROM_REGISTRY)
+        .defaultValue("15 secs")
+        .build();
+
+    public static final PropertyDescriptor BUCKET = new Builder()
+        .name("Registry Bucket")
+        .displayName("Registry Bucket")
+        .description("The name of the Bucket in the NiFi Registry that the flow should retrieved from")
+        .required(true)
+        .addValidator(NON_EMPTY_VALIDATOR)
+        .dependsOn(DATAFLOW_SPECIFICATION_STRATEGY, SPEC_FROM_REGISTRY)
+        .build();
+
+    public static final PropertyDescriptor FLOW_NAME = new Builder()
+        .name("Flow Name")
+        .displayName("Flow Name")
+        .description("The name of the flow in the NiFi Registry")
+        .required(true)
+        .addValidator(NON_EMPTY_VALIDATOR)
+        .dependsOn(DATAFLOW_SPECIFICATION_STRATEGY, SPEC_FROM_REGISTRY)
+        .build();
+
+    public static final PropertyDescriptor FLOW_VERSION = new Builder()
+        .name("Flow Version")
+        .displayName("Flow Version")
+        .description("The version of the flow in the NiFi Registry that should be retrieved. If not specified, the latest version will always be used.")
+        .required(false)
+        .addValidator(POSITIVE_INTEGER_VALIDATOR)
+        .dependsOn(DATAFLOW_SPECIFICATION_STRATEGY, SPEC_FROM_REGISTRY)
+        .build();
+
+    public static final PropertyDescriptor INPUT_PORT = new Builder()
+        .name("Input Port")
+        .displayName("Input Port")
+        .description("Specifies the name of the Input Port to send incoming FlowFiles to. This property is required if this processor has any incoming connections.")
+        .required(false)
+        .addValidator(NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(FLOWFILE_ATTRIBUTES)
+        .build();
+
+    public static final PropertyDescriptor FAILURE_PORTS = new Builder()
+        .name("Failure Ports")
+        .displayName("Failure Ports")
+        .description("A comma-separated list of the names of Output Ports that exist at the root level of the dataflow. If any FlowFile is routed to one of the Ports whose name is listed here, the " +
+            "dataflow will be considered a failure, and the incoming FlowFile (if any) will be routed to 'failure'. If not specified, all Output Ports will be considered successful.")
+        .required(false)
+        .addValidator(NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(NONE)
+        .build();
+
+    public static final PropertyDescriptor DATAFLOW_TIMEOUT = new Builder()
+        .name("Dataflow Timeout")
+        .displayName("Dataflow Timeout")
+        .description("If the flow does not complete within this amount of time, the incoming FlowFile, if any, will be routed to the timeout relationship," +
+            "the dataflow will be cancelled, and the invocation will end.")
+        .required(true)
+        .addValidator(TIME_PERIOD_VALIDATOR)
+        .expressionLanguageSupported(FLOWFILE_ATTRIBUTES)
+        .defaultValue("60 sec")
+        .build();
+
+    public static final PropertyDescriptor LIB_DIRECTORY = new Builder()
+        .name("NAR Directory")
+        .displayName("NAR Directory")
+        .description("The directory to retrieve NAR's from")
+        .required(true)
+        .addValidator(createDirectoryExistsValidator(false, false))
+        .defaultValue("./lib")
+        .build();
+
+    public static final PropertyDescriptor WORKING_DIRECTORY = new Builder()
+        .name("Work Directory")
+        .displayName("Work Directory")
+        .description("A directory that can be used to create temporary files, such as expanding NAR files, temporary FlowFile content, caching the dataflow, etc.")
+        .required(true)
+        .addValidator(createDirectoryExistsValidator(false, true))
+        .defaultValue("./work")
+        .build();
+
+    public static final PropertyDescriptor KRB5_CONF = new Builder()
+        .name("Krb5 Conf File")
+        .displayName("Krb5 Conf File")
+        .description("The KRB5 Conf file to use for configuring components that rely on Kerberos")
+        .required(false)
+        .identifiesExternalResource(ResourceCardinality.SINGLE, ResourceType.FILE)
+        .build();
+
+    public static final PropertyDescriptor STATELESS_SSL_CONTEXT_SERVICE = new Builder()
+        .name("Stateless SSL Context Service")
+        .displayName("Stateless SSL Context Service")
+        .description("The SSL Context to use as the Stateless System SSL Context")
+        .required(false)
+        .identifiesControllerService(SSLContextService.class)
+        .build();
+
+    public static final PropertyDescriptor MAX_INGEST_FLOWFILES = new Builder()
+        .name("Max Ingest FlowFiles")
+        .displayName("Max Ingest FlowFiles")
+        .description("During the course of a stateless dataflow, some processors may require more data than they have available in order to proceed. For example, MergeContent may require a minimum " +
+            "number of FlowFiles before it can proceed. In this case, the dataflow may bring in additional data from its source Processor. However, this data may all be held in memory, so this " +
+            "property provides a mechanism for limiting the maximum number of FlowFiles that the source Processor can ingest before it will no longer be triggered to ingest additional data.")
+        .required(false)
+        .addValidator(POSITIVE_INTEGER_VALIDATOR)
+        .expressionLanguageSupported(NONE)
+        .build();
+
+    public static final PropertyDescriptor MAX_INGEST_DATA_SIZE = new Builder()
+        .name("Max Ingest Data Size")
+        .displayName("Max Ingest Data Size")
+        .description("During the course of a stateless dataflow, some processors may require more data than they have available in order to proceed. For example, MergeContent may require a minimum " +
+            "number of FlowFiles before it can proceed. In this case, the dataflow may bring in additional data from its source Processor. However, this data may all be held in memory, so this " +
+            "property provides a mechanism for limiting the maximum amount of data that the source Processor can ingest before it will no longer be triggered to ingest additional data.")
+        .required(false)
+        .addValidator(DATA_SIZE_VALIDATOR)
+        .expressionLanguageSupported(NONE)
+        .build();
+
+    public static final PropertyDescriptor CONTENT_STORAGE_STRATEGY = new Builder()
+        .name("Content Storage Strategy")
+        .displayName("Content Storage Strategy")
+        .description("Specifies where the content of FlowFiles that the Stateless dataflow is operating on should be stored. Note that the data is always considered temporary and may be deleted at " +
+            "any time. It is not intended to be persisted across restarted.")
+        .required(true)
+        .allowableValues(CONTENT_STORAGE_HEAP, CONTENT_STORAGE_DISK)
+        .defaultValue(CONTENT_STORAGE_DISK.getValue())
+        .build();
+
+    public static final PropertyDescriptor MAX_INPUT_FLOWFILE_SIZE = new Builder()
+        .name("Max Input FlowFile Size")
+        .displayName("Max Input FlowFile Size")
+        .description("This Processor is configured to load all incoming FlowFiles into memory. Because of that, it is important to limit the maximum size of " +
+            "any incoming FlowFile that would get loaded into memory, in order to prevent Out Of Memory Errors and excessive Garbage Collection. Any FlowFile whose content " +
+            "size is greater than the configured size will be routed to failure and not sent to the Stateless Engine.")
+        .required(true)
+        .dependsOn(CONTENT_STORAGE_STRATEGY, CONTENT_STORAGE_HEAP)
+        .addValidator(DATA_SIZE_VALIDATOR)
+        .expressionLanguageSupported(NONE)
+        .defaultValue("1 MB")
+        .build();
+
+    static final Relationship REL_ORIGINAL = new Relationship.Builder()
+        .name("original")
+        .description("For any incoming FlowFile that is successfully processed, the original incoming FlowFile will be transferred to this Relationship")
+        .autoTerminateDefault(true)
+        .build();
+    static final Relationship REL_OUTPUT = new Relationship.Builder()
+        .name("output")
+        .description("Any FlowFiles that are transferred to an Output Port in the configured dataflow will be routed to this Relationship")
+        .build();
+    static final Relationship REL_FAILURE = new Relationship.Builder()
+        .name("failure")
+        .description("If the dataflow fails to process an incoming FlowFile, that FlowFile will be routed to this relationship")
+        .build();
+    static final Relationship REL_TIMEOUT = new Relationship.Builder()
+        .name("timeout")
+        .description("If the dataflow fails to complete in the configured amount of time, any incoming FlowFile will be routed to this relationship")
+        .build();
+
+
+    private final BlockingQueue<StatelessDataflow> dataflows = new LinkedBlockingDeque<>();
+    private final AtomicInteger dataflowCreationCount = new AtomicInteger(0);
+    private volatile Set<String> failurePortNames;
+    private volatile VersionedFlowSnapshot flowSnapshot;
+    private volatile AbortableTriggerContext triggerContext;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return Arrays.asList(
+            DATAFLOW_SPECIFICATION_STRATEGY,
+            DATAFLOW_FILE,
+            REGISTRY_URL,
+            SSL_CONTEXT_SERVICE,
+            COMMS_TIMEOUT,
+            BUCKET,
+            FLOW_NAME,
+            FLOW_VERSION,
+            INPUT_PORT,
+            FAILURE_PORTS,
+            CONTENT_STORAGE_STRATEGY,
+            MAX_INPUT_FLOWFILE_SIZE,
+            DATAFLOW_TIMEOUT,
+            LIB_DIRECTORY,
+            WORKING_DIRECTORY,
+            MAX_INGEST_FLOWFILES,
+            MAX_INGEST_DATA_SIZE,
+            STATELESS_SSL_CONTEXT_SERVICE,
+            KRB5_CONF);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return new HashSet<>(Arrays.asList(REL_ORIGINAL, REL_OUTPUT, REL_FAILURE, REL_TIMEOUT));
+    }
+
+    @Override
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+        return new Builder()
+            .name(propertyDescriptorName)
+            .defaultValue("Value for the " + propertyDescriptorName + " parameter")
+            .addValidator(Validator.VALID)
+            .sensitive(true)
+            .dynamic(true)
+            .build();
+    }
+
+
+    @OnScheduled
+    public void parseDataflow(final ProcessContext context) throws IOException {
+        final String specificationStrategy = context.getProperty(DATAFLOW_SPECIFICATION_STRATEGY).getValue();
+
+        final DataflowProvider rawRetrieval;
+        if (specificationStrategy.equalsIgnoreCase(SPEC_FROM_FILE.getValue())) {
+            rawRetrieval = new FileSystemDataflowProvider();
+        } else {
+            rawRetrieval = new RegistryDataflowProvider(getLogger());
+        }
+
+        final DataflowProvider cachedRetrieval = new CachingDataflowProvider(getIdentifier(), getLogger(), rawRetrieval);
+
+        final long start = System.nanoTime();
+        final VersionedFlowSnapshot versionedFlowSnapshot = cachedRetrieval.retrieveDataflowContents(context);
+        final long millis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start);
+
+        getLogger().info("Successfully retrieved flow in {} millis", millis);
+        this.flowSnapshot = versionedFlowSnapshot;
+        this.triggerContext = new AbortableTriggerContext();
+
+        final Set<String> failurePorts = new HashSet<>();
+        final String failurePortNames = context.getProperty(FAILURE_PORTS).getValue();
+        if (failurePortNames != null) {
+            for (final String portName : failurePortNames.split(",")) {
+                failurePorts.add(portName.trim());
+            }
+        }
+
+        this.failurePortNames = failurePorts;
+    }
+
+    @OnUnscheduled
+    public void abortDataflow() {
+        if (triggerContext != null) {
+            triggerContext.abort();
+        }
+    }
+
+    @OnStopped
+    public void shutdown() {
+        StatelessDataflow dataflow;
+        while ((dataflow = dataflows.poll()) != null) {
+            dataflow.shutdown();
+        }
+
+        dataflows.clear();
+        dataflowCreationCount.set(0);
+    }
+
+    private StatelessDataflow createDataflow(final ProcessContext context) throws IOException, StatelessConfigurationException {
+        final int dataflowIndex = dataflowCreationCount.getAndIncrement();
+        final StatelessEngineConfiguration engineConfiguration = createEngineConfiguration(context, dataflowIndex);
+        final StatelessBootstrap bootstrap = StatelessBootstrap.bootstrap(engineConfiguration, Thread.currentThread().getContextClassLoader());
+
+        final DataflowDefinition<VersionedFlowSnapshot> dataflowDefinition = createDataflowDefinition(context, flowSnapshot);
+
+        final StatelessDataflow dataflow = bootstrap.createDataflow(dataflowDefinition);
+        dataflow.initialize();
+        return dataflow;
+    }
+
+    private StatelessDataflow getDataflow(final ProcessContext context) throws IOException, StatelessConfigurationException {
+        final StatelessDataflow dataflow = dataflows.poll();
+        if (dataflow == null) {
+            return createDataflow(context);
+        }
+
+        return dataflow;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        // Fetch a FlowFile, if appropriate
+        FlowFile flowFile = null;
+        if (context.hasIncomingConnection()) {
+            flowFile = session.get();
+            if (flowFile == null) {
+                return;
+            }
+        }
+
+        // Get the dataflow to run
+        final StatelessDataflow dataflow;
+        try {
+            dataflow = getDataflow(context);
+        } catch (final Exception e) {
+            getLogger().error("Could not create dataflow from snapshot", e);
+            session.rollback();
+            return;
+        }
+
+        // Trigger the dataflow and make sure that we always add the StatelessDataflow object back to the queue so that it can be reused.
+        try {
+            runDataflow(dataflow, flowFile, context, session);
+        } finally {
+            dataflows.offer(dataflow);
+        }
+    }
+
+
+    private void runDataflow(final StatelessDataflow dataflow, final FlowFile flowFile, final ProcessContext context, final ProcessSession session) {
+        // Ensure that we get a legitimate timeout value
+        final long timeoutMillis;
+        try {
+            timeoutMillis = context.getProperty(DATAFLOW_TIMEOUT).evaluateAttributeExpressions(flowFile).asTimePeriod(TimeUnit.MILLISECONDS);
+        } catch (final Exception e) {
+            getLogger().error("Failed to determine Dataflow Timeout for {}. Routing to failure", flowFile);
+            session.transfer(flowFile, REL_FAILURE);
+            return;
+        }
+
+        // Attempt to enqueue the dataflow. If unable, the appropriate log messages will be generated and actions taken by the enqueueFlowFile() method, so we can simply return.
+        if (flowFile != null) {
+            final boolean enqueued = enqueueFlowFile(flowFile, dataflow, context, session);
+            if (!enqueued) {
+                return;
+            }
+        }
+
+        // Reset any counters on the dataflow. This way, we can simply gather the counters after triggering the dataflow,
+        // and we know that's how much we need to adjust our counters by.
+        dataflow.resetCounters();
+
+        // Trigger the dataflow
+        final BulletinRepository bulletinRepository = dataflow.getBulletinRepository();
+        final long maxBulletinId = bulletinRepository.getMaxBulletinId();
+        final DataflowTrigger trigger = dataflow.trigger(triggerContext);
+
+        // If the timeout is exceeded, transfer original FlowFile to failure and cancel the dataflow invocation.
+        Optional<TriggerResult> optionalResult;
+        boolean timeoutExceeded = false;
+        try {
+            optionalResult = trigger.getResult(timeoutMillis, TimeUnit.MILLISECONDS);
+        } catch (final InterruptedException e) {
+            Thread.currentThread().interrupt();
+            timeoutExceeded = true;
+            optionalResult = Optional.empty();
+            trigger.cancel();
+        } finally {
+            surfaceBulletins(bulletinRepository, maxBulletinId);
+        }
+
+        if (!optionalResult.isPresent()) {
+            timeoutExceeded = true;
+        }
+
+        if (timeoutExceeded) {
+            getLogger().error("Dataflow did not complete within the allotted time of {} milliseconds for {}. Routing to timeout.", timeoutMillis, flowFile);
+            if (flowFile != null) {
+                session.transfer(flowFile, REL_TIMEOUT);
+            }
+            trigger.cancel();
+            return;
+        }
+
+        // If the datflow was not successful, log an indication of why not and transfer to failure. Then return, as the dataflow has completed and there's nothing left to do.
+        final TriggerResult triggerResult = optionalResult.get();
+        if (!triggerResult.isSuccessful()) {
+            final Optional<Throwable> failureOptional = triggerResult.getFailureCause();
+            if (failureOptional.isPresent()) {
+                final Throwable cause = failureOptional.get();
+
+                if (flowFile == null) {
+                    getLogger().error("Dataflow failed to complete successfully. Yielding.", failureOptional.get());
+                } else {
+                    getLogger().error("Dataflow failed to complete successfully for {}. Routing to failure and yielding.", flowFile, failureOptional.get());
+
+                    // Add a failure.port.name attribute
+                    if (cause instanceof FailurePortEncounteredException) {
+                        final String portName = ((FailurePortEncounteredException) cause).getPortName();
+                        session.putAttribute(flowFile, "failure.port.name", portName);
+                    }
+                }
+
+            }
+
+            if (flowFile != null) {
+                session.transfer(flowFile, REL_FAILURE);
+            }
+
+            adjustCounters(session, dataflow, " (Failed attempts)");
+            session.adjustCounter("Failed Invocations", 1, false);
+            context.yield();
+
+            return;
+        }
+
+        // Create a FlowFile in this NiFi instance for each FlowFile that was output by the Stateless dataflow.
+        // We cannot simply transfer the output FlowFiles because they belong to a different, internal session and their content may not be persisted.
+        // Therefore, we create our own FlowFile whose parent is the input FlowFile (if one exists) and then add the attributes and contents as necessary.
+        final Set<FlowFile> createdSet;
+        try {
+            createdSet = createOutputFlowFiles(optionalResult.get(), session, flowFile);
+        } catch (final IOException e) {
+            getLogger().error("Failed to write FlowFile contents that were output from Stateless Flow to the NiFi content repository for {}. Routing to failure.", flowFile, e);
+            if (flowFile != null) {
+                session.transfer(flowFile, REL_FAILURE);
+            }
+            return;
+        }
+
+        // Update any counters
+        adjustCounters(session, dataflow, null);
+
+        // If dataflow is yielded, yield this processor
+        final long yieldExpiration = dataflow.getSourceYieldExpiration();
+        if (yieldExpiration > 0) {
+            final long now = System.currentTimeMillis();
+            final long yieldMillis = yieldExpiration - now;
+            if (yieldMillis > 0) {
+                context.yield();
+            }
+        }
+
+        // Transfer the FlowFiles and asynchronously commit the session.
+        if (flowFile != null) {
+            session.transfer(flowFile, REL_ORIGINAL);
+        }
+        session.transfer(createdSet, REL_OUTPUT);
+        session.commitAsync(triggerResult::acknowledge);
+
+        if (flowFile == null) {
+            getLogger().info("Successfully triggered dataflow to run, producing {} output FlowFiles", createdSet.size());
+        } else {
+            getLogger().info("Successfully triggered dataflow to run against {}, producing {} output FlowFiles", flowFile, createdSet.size());
+        }
+
+        session.adjustCounter("Successful Invocations", 1, false);
+    }
+
+    private void surfaceBulletins(final BulletinRepository bulletinRepository, final long minBulletinId) {
+        // If there are any WARNING or ERROR bulletins, we want to log them for this processor. All of the log messages from the components
+        // themselves will already have been logged, but we want to surface any warn/error message as bulletins so we log them again for this processor.
+        final BulletinQuery bulletinQuery = new BulletinQuery.Builder()
+            .after(minBulletinId)
+            .build();
+
+        final List<Bulletin> bulletins = bulletinRepository.findBulletins(bulletinQuery);
+        for (final Bulletin bulletin : bulletins) {
+            try {
+                String level = bulletin.getLevel();
+                if (level == null || level.equalsIgnoreCase("WARNING")) {
+                    level = "WARN";
+                }
+                final LogLevel logLevel = LogLevel.valueOf(level);
+                if (logLevel == LogLevel.DEBUG || logLevel == LogLevel.INFO) {
+                    continue;
+                }
+
+                getLogger().log(logLevel, "{} {}[name={}, id={}] {}", bulletin.getTimestamp(), bulletin.getSourceType(), bulletin.getSourceName(), bulletin.getSourceName(), bulletin.getMessage());
+            } catch (final Exception e) {
+                getLogger().warn("Dataflow emitted a bulletin but failed to surface that bulletin due to {}", e.toString(), e);
+            }
+        }
+    }
+
+    private void adjustCounters(final ProcessSession session, final StatelessDataflow dataflow, final String counterNameSuffix) {
+        for (final Map.Entry<String, Long> entry : dataflow.getCounters(false).entrySet()) {
+            if (entry.getValue() != 0) {
+                final String counterName = counterNameSuffix == null ? entry.getKey() : (entry.getKey() + counterNameSuffix);
+                session.adjustCounter(counterName, entry.getValue(), false);
+            }
+        }
+    }
+
+    private Set<FlowFile> createOutputFlowFiles(final TriggerResult triggerResult, final ProcessSession session, final FlowFile flowFile) throws IOException {
+        final Set<FlowFile> createdSet = new HashSet<>();
+        try {
+            final Map<String, List<FlowFile>> outputFlowFiles = triggerResult.getOutputFlowFiles();
+            for (final Map.Entry<String, List<FlowFile>> entry : outputFlowFiles.entrySet()) {
+                final String outputPortName = entry.getKey();
+                final List<FlowFile> outputForPort = entry.getValue();
+
+                for (final FlowFile outputFlowFile : outputForPort) {
+                    FlowFile created = flowFile == null ? session.create() : session.create(flowFile);
+                    createdSet.add(created);
+
+                    try (final OutputStream out = session.write(created);
+                         final InputStream flowFileContents = triggerResult.readContent(outputFlowFile)) {
+                        StreamUtils.copy(flowFileContents, out);
+                    }
+
+                    final Map<String, String> attributes = new HashMap<>(outputFlowFile.getAttributes());
+                    attributes.put("output.port.name", outputPortName);
+                    session.putAllAttributes(created, attributes);
+                }
+            }
+        } catch (final Exception e) {
+            session.remove(createdSet);
+            throw e;
+        }
+
+        return createdSet;
+    }
+
+    private boolean enqueueFlowFile(final FlowFile flowFile, final StatelessDataflow dataflow, final ProcessContext context, final ProcessSession session) {
+        final long maxBytes = context.getProperty(MAX_INPUT_FLOWFILE_SIZE).asDataSize(DataUnit.B).longValue();
+        if (flowFile.getSize() > maxBytes) {
+            getLogger().warn("Will not process {} because its size of {} bytes exceeds the max configured threshold of {} bytes. Routing to failure",
+                flowFile, flowFile.getSize(), maxBytes);
+
+            session.transfer(flowFile, REL_FAILURE);
+            return false;
+        }
+
+        final Set<String> inputPortNames = dataflow.getInputPortNames();
+
+        // If there is exactly 1 Input Port available in the dataflow, the name isn't required.
+        String inputPortName = context.getProperty(INPUT_PORT).evaluateAttributeExpressions(flowFile).getValue();
+        if (inputPortName == null || inputPortName.trim().isEmpty()) {
+            if (inputPortNames.size() == 1) {
+                inputPortName = inputPortNames.iterator().next();
+            } else {
+                getLogger().error("For {}, determined Input Port Name to be unspecified. Routing to failure.", flowFile);
+                session.transfer(flowFile, REL_FAILURE);
+                return false;
+            }
+        }
+
+        if (!inputPortNames.contains(inputPortName)) {
+            getLogger().error("For {}, Input Port Name is {}, but that Input Port does not exist in the provided dataflow or is not at the root level. Routing to failure",
+                flowFile, inputPortName);
+            session.transfer(flowFile, REL_FAILURE);
+            return false;
+        }
+
+        try (final InputStream in = session.read(flowFile)) {
+            dataflow.enqueue(in, flowFile.getAttributes(), inputPortName);
+        } catch (final IOException e) {
+            getLogger().error("Failed to read contents of FlowFile {} into memory. Routing to failure", flowFile);
+            session.transfer(flowFile, REL_FAILURE);
+            return false;
+        }
+
+        return true;
+    }
+
+
+    private DataflowDefinition<VersionedFlowSnapshot> createDataflowDefinition(final ProcessContext context, final VersionedFlowSnapshot flowSnapshot) {
+        final ParameterValueProviderDefinition parameterValueProviderDefinition = new ParameterValueProviderDefinition();
+        parameterValueProviderDefinition.setType("org.apache.nifi.stateless.parameter.OverrideParameterValueProvider");
+        parameterValueProviderDefinition.setName("Parameter Override");
+
+        final Map<String, String> parameterValues = new HashMap<>();
+        for (final Map.Entry<PropertyDescriptor, String> entry : context.getProperties().entrySet()) {
+            if (!entry.getKey().isDynamic()) {
+                continue;
+            }
+
+            parameterValues.put(entry.getKey().getName(), entry.getValue());
+        }
+
+        parameterValueProviderDefinition.setPropertyValues(parameterValues);
+
+        final Integer maxFlowFiles = context.getProperty(MAX_INGEST_FLOWFILES).asInteger();
+        final Double maxBytes = context.getProperty(MAX_INGEST_DATA_SIZE).asDataSize(DataUnit.B);
+        final long maxTimeNanos = context.getProperty(DATAFLOW_TIMEOUT).asTimePeriod(TimeUnit.NANOSECONDS);
+
+        final TransactionThresholds transactionThresholds = new TransactionThresholds() {
+            @Override
+            public OptionalLong getMaxFlowFiles() {
+                return maxFlowFiles == null ? OptionalLong.empty() : OptionalLong.of(maxFlowFiles);
+            }
+
+            @Override
+            public OptionalLong getMaxContentSize(final DataUnit dataUnit) {
+                return maxBytes == null ? OptionalLong.empty() : OptionalLong.of(maxBytes.longValue());
+            }
+
+            @Override
+            public OptionalLong getMaxTime(final TimeUnit timeUnit) {
+                return OptionalLong.of(timeUnit.convert(maxTimeNanos, TimeUnit.NANOSECONDS));
+            }
+        };
+
+        return new DataflowDefinition<VersionedFlowSnapshot>() {
+            @Override
+            public VersionedFlowSnapshot getFlowSnapshot() {
+                return flowSnapshot;
+            }
+
+            @Override
+            public String getFlowName() {
+                return flowSnapshot.getFlowContents().getName();
+            }
+
+            @Override
+            public Set<String> getFailurePortNames() {
+                return failurePortNames;
+            }
+
+            @Override
+            public List<ParameterContextDefinition> getParameterContexts() {
+                return null;
+            }
+
+            @Override
+            public List<ReportingTaskDefinition> getReportingTaskDefinitions() {
+                return Collections.emptyList();
+            }
+
+            @Override
+            public List<ParameterValueProviderDefinition> getParameterValueProviderDefinitions() {
+                return Collections.singletonList(parameterValueProviderDefinition);
+            }
+
+            @Override
+            public TransactionThresholds getTransactionThresholds() {
+                return transactionThresholds;
+            }
+        };
+    }
+
+    private StatelessEngineConfiguration createEngineConfiguration(final ProcessContext context, final int contentRepoIndex) {
+        final File workingDirectory = new File(context.getProperty(WORKING_DIRECTORY).getValue());
+        final File narDirectory = new File(context.getProperty(LIB_DIRECTORY).getValue());
+        final ResourceReference krb5Reference = context.getProperty(KRB5_CONF).asResource();
+        final File krb5Conf = krb5Reference == null ? null : krb5Reference.asFile();
+        final SSLContextService sslContextService = context.getProperty(STATELESS_SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
+
+        final SslContextDefinition sslContextDefinition;
+        if (sslContextService == null) {
+            sslContextDefinition = null;
+        } else {
+            sslContextDefinition = new SslContextDefinition();
+            sslContextDefinition.setKeyPass(sslContextService.getKeyPassword());
+            sslContextDefinition.setKeystoreFile(sslContextService.getKeyStoreFile());
+            sslContextDefinition.setKeystorePass(sslContextService.getKeyStorePassword());
+            sslContextDefinition.setKeystoreType(sslContextService.getKeyStoreType());
+            sslContextDefinition.setTruststoreFile(sslContextService.getTrustStoreFile());
+            sslContextDefinition.setTruststorePass(sslContextService.getTrustStorePassword());
+            sslContextDefinition.setTruststoreType(sslContextService.getTrustStoreType());
+        }
+
+        final String contentStorageStrategy = context.getProperty(CONTENT_STORAGE_STRATEGY).getValue();
+        final File contentRepoDirectory;
+        if (CONTENT_STORAGE_DISK.getValue().equals(contentStorageStrategy)) {
+            final File contentRepoRootDirectory = new File(workingDirectory, "execute-stateless-flowfile-content");
+            final File processorContentRepo = new File(contentRepoRootDirectory, getIdentifier());
+            contentRepoDirectory = new File(processorContentRepo, String.valueOf(contentRepoIndex));
+        } else {
+            contentRepoDirectory = null;
+        }
+
+        return new StatelessEngineConfiguration() {
+            @Override
+            public File getWorkingDirectory() {
+                return workingDirectory;
+            }
+
+            @Override
+            public File getNarDirectory() {
+                return narDirectory;
+            }
+
+            @Override
+            public File getExtensionsDirectory() {
+                return narDirectory;
+            }
+
+            @Override
+            public File getKrb5File() {
+                return krb5Conf;
+            }
+
+            @Override
+            public Optional<File> getContentRepositoryDirectory() {
+                return Optional.ofNullable(contentRepoDirectory);
+            }
+
+            @Override
+            public SslContextDefinition getSslContext() {
+                return sslContextDefinition;
+            }
+
+            @Override
+            public String getSensitivePropsKey() {
+                return getIdentifier();
+            }
+
+            @Override
+            public List<ExtensionClientDefinition> getExtensionClients() {
+                return Collections.emptyList();
+            }
+
+            @Override
+            public boolean isLogExtensionDiscovery() {
+                return false;
+            }
+        };
+    }
+
+    @Override
+    public Collection<SearchResult> search(final SearchContext context) {
+        if (flowSnapshot == null) {
+            return Collections.emptyList();
+        }
+
+        final VersionedComponentSearchResults results = new VersionedComponentSearchResults(context.getSearchTerm());
+        final Bucket bucket = flowSnapshot.getBucket();
+        if (bucket != null) {
+            results.add(bucket.getIdentifier(), "Bucket ID");
+            results.add(bucket.getName(), "Bucket Name");
+            results.add(bucket.getDescription(), "Bucket Description");
+        }
+
+        final VersionedFlow versionedFlow = flowSnapshot.getFlow();
+        if (versionedFlow != null) {
+            results.add(versionedFlow.getIdentifier(), "Flow ID");
+            results.add(versionedFlow.getName(), "Flow Name");
+            results.add(versionedFlow.getDescription(), "Flow Description");
+        }
+
+        search(flowSnapshot.getFlowContents(), results);
+        return results.toList();
+    }
+
+    private void search(final VersionedProcessGroup group, final VersionedComponentSearchResults results) {
+        results.add(group.getName(), "Process Group Name");
+        results.add(group.getComments(), "Process Group Comments");
+
+        for (final VersionedPort port : group.getInputPorts()) {
+            results.add(port.getName(), "Input Port Name");
+            results.add(port.getComments(), "Input Port Comments");
+            results.add(port.getIdentifier(), "Input Port ID");
+        }
+        for (final VersionedPort port : group.getOutputPorts()) {
+            results.add(port.getName(), "Output Port Name");
+            results.add(port.getComments(), "Output Port Comments");
+            results.add(port.getIdentifier(), "Output Port ID");
+        }
+        for (final VersionedLabel label : group.getLabels()) {
+            results.add(label.getLabel(), "Label Text");
+        }
+        for (final VersionedProcessor processor : group.getProcessors()) {
+            results.add(processor.getName(), "Processor Name");
+            results.add(processor.getType(), "Processor Type");
+            results.add(processor.getIdentifier(), "Processor ID");
+
+            for (final Map.Entry<String, String> entry : processor.getProperties().entrySet()) {
+                results.add(entry.getKey(), "Processor Property Name");
+                results.add(entry.getValue(), "Value of Processor Property " + entry.getKey());
+            }
+            results.add(processor.getComments(), "Processor Comments");
+
+            final Bundle bundle = processor.getBundle();
+            if (bundle != null) {
+                results.add(bundle.getGroup(), "Bundle Group ID for Processor " + processor.getType());
+                results.add(bundle.getArtifact(), "Bundle Artifact ID for Processor " + processor.getType());
+                results.add(bundle.getVersion(), "Bundle Version for Processor " + processor.getType());
+            }
+        }
+        for (final VersionedRemoteProcessGroup rpg : group.getRemoteProcessGroups()) {
+            results.add(rpg.getTargetUris(), "RPG Target URI");
+            results.add(rpg.getComments(), "RPG Comments");
+            results.add(rpg.getIdentifier(), "RPG Identifier");
+
+            for (final VersionedRemoteGroupPort port : rpg.getInputPorts()) {
+                results.add(port.getName(), "RPG Input Port Name");
+                results.add(port.getIdentifier(), "RPG Input Port ID");
+                results.add(port.getTargetId(), "RPG Input Port Target ID");
+            }
+
+            for (final VersionedRemoteGroupPort port : rpg.getOutputPorts()) {
+                results.add(port.getName(), "RPG Output Port Name");
+                results.add(port.getIdentifier(), "RPG Output Port ID");
+                results.add(port.getTargetId(), "RPG Output Port Target ID");
+            }
+        }
+        for (final Map.Entry<String, String> entry : group.getVariables().entrySet()) {
+            results.add(entry.getKey(), "Variable Name");
+            results.add(entry.getValue(), "Value of Variable " + entry.getKey());
+        }
+        results.add(group.getParameterContextName(), "Parameter Context Name");
+
+        for (final VersionedConnection connection : group.getConnections()) {
+            results.add(connection.getIdentifier(), "Connection ID");
+            results.add(connection.getName(), "Connection Name");
+            if (connection.getSelectedRelationships() != null) {
+                results.add(connection.getSelectedRelationships().toString(), "Selected Relationships");
+            }
+            results.add(connection.getComments(), "Connection Comments");
+        }
+        for (final VersionedControllerService service : group.getControllerServices()) {
+            results.add(service.getName(), "Controller Service Name");
+            results.add(service.getType(), "Controller Service Type");
+            results.add(service.getIdentifier(), "Controller Service ID");
+
+            for (final Map.Entry<String, String> entry : service.getProperties().entrySet()) {
+                results.add(entry.getKey(), "Controller Service Property Name");
+                results.add(entry.getValue(), "Value of Controller Service Property " + entry.getKey());
+            }
+            results.add(service.getComments(), "Controller Service Comments");
+
+            final Bundle bundle = service.getBundle();
+            if (bundle != null) {
+                results.add(bundle.getGroup(), "Bundle Group ID for Controller Service " + service.getType());
+                results.add(bundle.getArtifact(), "Bundle Artifact ID for Controller Service " + service.getType());
+                results.add(bundle.getVersion(), "Bundle Version for Controller Service " + service.getType());
+            }
+        }
+        for (final VersionedProcessGroup child : group.getProcessGroups()) {
+            search(child, results);
+        }
+    }
+
+
+    private static class VersionedComponentSearchResults {
+        private final String term;
+        private final List<SearchResult> results = new ArrayList<>();
+
+        public VersionedComponentSearchResults(final String term) {
+            this.term = term;
+        }
+
+        public void add(final String value, final String description) {
+            if (value == null) {
+                return;
+            }
+
+            if (value.contains(term)) {
+                results.add(new SearchResult.Builder().match(value).label(description).build());
+            }
+        }
+
+        public List<SearchResult> toList() {
+            return results;
+        }
+    }
+
+    private static class AbortableTriggerContext implements DataflowTriggerContext {
+        private volatile boolean aborted = false;
+
+        @Override
+        public boolean isAbort() {
+            return aborted;
+        }
+
+        public void abort() {
+            this.aborted = true;
+        }
+    }
+}
diff --git a/nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/retrieval/CachingDataflowProvider.java b/nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/retrieval/CachingDataflowProvider.java
new file mode 100644
index 0000000..29a1877
--- /dev/null
+++ b/nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/retrieval/CachingDataflowProvider.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.processors.stateless.retrieval;
+
+import com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.module.jaxb.JaxbAnnotationIntrospector;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processors.stateless.ExecuteStateless;
+import org.apache.nifi.registry.flow.VersionedFlowSnapshot;
+
+import java.io.File;
+import java.io.IOException;
+
+/**
+ * Wrapper that can be used in order to cache a dataflow once it has been fetched for backup purposes.
+ * This provider will always first delegate to the given DataflowProvider first. If the given provider is
+ * able to retrieve the dataflow, this provider will then store the dataflow is a file. If, later, the
+ * given provider is unable to retrieve the dataflow, due to the endpoint being down, etc., then this provider
+ * will instead parse the cached file. This eliminates the concern of requiring that some external endpoint is
+ * available in order to run the dataflow.
+ */
+public class CachingDataflowProvider implements DataflowProvider {
+    private final String processorId;
+    private final ComponentLog logger;
+    private final DataflowProvider delegate;
+    private final ObjectMapper objectMapper;
+
+
+    public CachingDataflowProvider(final String processorId, final ComponentLog logger, final DataflowProvider delegate) {
+        this.processorId = processorId;
+        this.logger = logger;
+        this.delegate = delegate;
+
+        objectMapper = new ObjectMapper();
+        objectMapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
+        objectMapper.setAnnotationIntrospector(new JaxbAnnotationIntrospector(objectMapper.getTypeFactory()));
+    }
+
+    @Override
+    public VersionedFlowSnapshot retrieveDataflowContents(final ProcessContext context) throws IOException {
+        try {
+            final VersionedFlowSnapshot retrieved = delegate.retrieveDataflowContents(context);
+            cacheFlowSnapshot(context, retrieved);
+            return retrieved;
+        } catch (final Exception e) {
+            final File cacheFile = getFlowCacheFile(context, processorId);
+            if (cacheFile.exists()) {
+                logger.warn("Failed to retrieve Flow Snapshot. Will restore Flow Snapshot from cached version at {}", cacheFile.getAbsolutePath(), e);
+                return readCachedFlow(cacheFile);
+            }
+
+            throw new IOException("Failed to retrieve Flow Snapshot from configured endpoint and no cached version is available", e);
+        }
+    }
+
+    private void cacheFlowSnapshot(final ProcessContext context, final VersionedFlowSnapshot flowSnapshot) {
+        final File cacheFile = getFlowCacheFile(context, processorId);
+        if (!cacheFile.getParentFile().exists() && !cacheFile.getParentFile().mkdirs()) {
+            logger.warn("Fetched dataflow but cannot create directory {} in order to cache the dataflow. " +
+                "Upon restart, processor will not be able to function unless flow endpoint is available", cacheFile);
+            return;
+        }
+
+        try {
+            objectMapper.writeValue(cacheFile, flowSnapshot);
+        } catch (final Exception e) {
+            logger.warn("Fetched dataflow but failed to write the dataflow to disk at {} in order to cache the dataflow. " +
+                "Upon restart, processor will not be able to function unless flow endpoint is available", cacheFile, e);
+        }
+    }
+
+    protected File getFlowCacheFile(final ProcessContext context, final String processorId) {
+        final String workingDirName = context.getProperty(ExecuteStateless.WORKING_DIRECTORY).getValue();
+        final File workingDir = new File(workingDirName);
+        final File dataflowCache = new File(workingDir, "dataflow-cache");
+        final File flowSnapshotFile = new File(dataflowCache, processorId + ".flow.snapshot.json");
+        return flowSnapshotFile;
+    }
+
+    private VersionedFlowSnapshot readCachedFlow(final File cacheFile) throws IOException {
+        return objectMapper.readValue(cacheFile, VersionedFlowSnapshot.class);
+    }
+}
diff --git a/nifi-stateless/nifi-stateless-api/src/main/java/org/apache/nifi/stateless/flow/FailurePortEncounteredException.java b/nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/retrieval/DataflowProvider.java
similarity index 71%
copy from nifi-stateless/nifi-stateless-api/src/main/java/org/apache/nifi/stateless/flow/FailurePortEncounteredException.java
copy to nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/retrieval/DataflowProvider.java
index fe18907..bfc729d 100644
--- a/nifi-stateless/nifi-stateless-api/src/main/java/org/apache/nifi/stateless/flow/FailurePortEncounteredException.java
+++ b/nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/retrieval/DataflowProvider.java
@@ -15,12 +15,13 @@
  * limitations under the License.
  */
 
-package org.apache.nifi.stateless.flow;
+package org.apache.nifi.processors.stateless.retrieval;
 
-public class FailurePortEncounteredException extends RuntimeException {
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.registry.flow.VersionedFlowSnapshot;
 
-    public FailurePortEncounteredException(String message) {
-        super(message);
-    }
+import java.io.IOException;
 
+public interface DataflowProvider {
+    VersionedFlowSnapshot retrieveDataflowContents(ProcessContext context) throws IOException;
 }
diff --git a/nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/retrieval/FileSystemDataflowProvider.java b/nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/retrieval/FileSystemDataflowProvider.java
new file mode 100644
index 0000000..e655074
--- /dev/null
+++ b/nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/retrieval/FileSystemDataflowProvider.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.processors.stateless.retrieval;
+
+import com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processors.stateless.ExecuteStateless;
+import org.apache.nifi.registry.flow.VersionedFlowSnapshot;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+public class FileSystemDataflowProvider implements DataflowProvider {
+    @Override
+    public VersionedFlowSnapshot retrieveDataflowContents(final ProcessContext context) throws IOException {
+        final ObjectMapper objectMapper = new ObjectMapper();
+        objectMapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
+
+        try (final InputStream in = context.getProperty(ExecuteStateless.DATAFLOW_FILE).asResource().read()) {
+            final VersionedFlowSnapshot versionedFlowSnapshot = objectMapper.readValue(in, VersionedFlowSnapshot.class);
+            return versionedFlowSnapshot;
+        }
+    }
+}
diff --git a/nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/retrieval/RegistryDataflowProvider.java b/nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/retrieval/RegistryDataflowProvider.java
new file mode 100644
index 0000000..9b0f46d
--- /dev/null
+++ b/nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/retrieval/RegistryDataflowProvider.java
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.processors.stateless.retrieval;
+
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processors.stateless.ExecuteStateless;
+import org.apache.nifi.registry.bucket.Bucket;
+import org.apache.nifi.registry.client.NiFiRegistryClient;
+import org.apache.nifi.registry.client.NiFiRegistryClientConfig;
+import org.apache.nifi.registry.client.NiFiRegistryException;
+import org.apache.nifi.registry.client.impl.JerseyNiFiRegistryClient;
+import org.apache.nifi.registry.flow.VersionedFlow;
+import org.apache.nifi.registry.flow.VersionedFlowSnapshot;
+import org.apache.nifi.ssl.SSLContextService;
+
+import javax.net.ssl.SSLContext;
+import java.io.IOException;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.TimeUnit;
+
+public class RegistryDataflowProvider implements DataflowProvider {
+    private final ComponentLog logger;
+
+    public RegistryDataflowProvider(final ComponentLog logger) {
+        this.logger = logger;
+    }
+
+    @Override
+    public VersionedFlowSnapshot retrieveDataflowContents(final ProcessContext context) throws IOException {
+        final SSLContextService sslContextService = context.getProperty(ExecuteStateless.SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
+        final SSLContext sslContext = sslContextService == null ? null : sslContextService.createContext();
+
+        final String url = context.getProperty(ExecuteStateless.REGISTRY_URL).getValue();
+        final NiFiRegistryClientConfig clientConfig = new NiFiRegistryClientConfig.Builder()
+            .baseUrl(url)
+            .connectTimeout(context.getProperty(ExecuteStateless.COMMS_TIMEOUT).asTimePeriod(TimeUnit.MILLISECONDS).intValue())
+            .readTimeout(context.getProperty(ExecuteStateless.COMMS_TIMEOUT).asTimePeriod(TimeUnit.MILLISECONDS).intValue())
+            .sslContext(sslContext)
+            .build();
+
+        final NiFiRegistryClient client = new JerseyNiFiRegistryClient.Builder()
+            .config(clientConfig)
+            .build();
+
+        final VersionedFlowSnapshot versionedFlowSnapshot;
+        final String bucketName = context.getProperty(ExecuteStateless.BUCKET).getValue();
+        final String flowName = context.getProperty(ExecuteStateless.FLOW_NAME).getValue();
+        final Integer flowVersion = context.getProperty(ExecuteStateless.FLOW_VERSION).asInteger();
+        try {
+            final String bucketId = getBucketId(client, bucketName);
+            final String flowId = getFlowId(client, flowName, bucketId);
+
+            logger.debug("Attempting to fetch dataflow from Registry at URL {}, Bucket {}, Flow {}, flowVersion {}", url, bucketId, flowId, flowVersion == null ? "<Latest>" : flowVersion);
+            if (flowVersion == null) {
+                versionedFlowSnapshot = client.getFlowSnapshotClient().getLatest(bucketId, flowId);
+            } else {
+                versionedFlowSnapshot = client.getFlowSnapshotClient().get(bucketId, flowId, flowVersion);
+            }
+
+            logger.debug("Successfully fetched dataflow from Registry at URL {}, Bucket {}, Flow {}, flowVersion {}", url, bucketId, flowId, flowVersion == null ? "<Latest>" : flowVersion);
+        } catch (final NiFiRegistryException e) {
+            throw new IOException("Failed to retrieve Flow Snapshot from Registry", e);
+        }
+
+        return versionedFlowSnapshot;
+    }
+
+    private String getFlowId(final NiFiRegistryClient client, final String flowName, final String bucketId) throws IOException {
+        final List<VersionedFlow> versionedFlows;
+        try {
+            versionedFlows = client.getFlowClient().getByBucket(bucketId);
+        } catch (NiFiRegistryException e) {
+            throw new IOException("Could not retrieve list of Flows from NiFi Registry for Bucket ID " + bucketId);
+        }
+
+        for (final VersionedFlow versionedFlow : versionedFlows) {
+            if (flowName.equals(versionedFlow.getName())) {
+                return versionedFlow.getIdentifier();
+            }
+        }
+
+        throw new IOException("Could not find a flow with the name '" + flowName + "' within bucket with ID '" + bucketId + "' in the given Registry");
+    }
+
+    private String getBucketId(final NiFiRegistryClient client, final String bucketName) throws IOException {
+        try {
+            final List<Bucket> allBuckets = client.getBucketClient().getAll();
+            final Optional<Bucket> optionalBucket = allBuckets.stream().filter(bkt -> bkt.getName().equals(bucketName)).findAny();
+            if (!optionalBucket.isPresent()) {
+                throw new IOException("Could not find a bucket with the name '" + bucketName + "' in the given Registry");
+            }
+
+            return optionalBucket.get().getIdentifier();
+        } catch (NiFiRegistryException e) {
+            throw new IOException("Failed to fetch buckets from NiFi Registry", e);
+        }
+    }
+}
diff --git a/nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor b/nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
new file mode 100644
index 0000000..6355dc7
--- /dev/null
+++ b/nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
@@ -0,0 +1,16 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+org.apache.nifi.processors.stateless.ExecuteStateless
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/resources/docs/org.apache.nifi.processors.stateless.ExecuteStateless/additionalDetails.html b/nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/resources/docs/org.apache.nifi.processors.stateless.ExecuteStateless/additionalDetails.html
new file mode 100644
index 0000000..ef1faa3
--- /dev/null
+++ b/nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/resources/docs/org.apache.nifi.processors.stateless.ExecuteStateless/additionalDetails.html
@@ -0,0 +1,358 @@
+<!DOCTYPE html>
+<html lang="en">
+<!--
+  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.
+-->
+<head>
+    <meta charset="utf-8"/>
+    <title>ExecuteStateless</title>
+    <link rel="stylesheet" href="../../../../../css/component-usage.css" type="text/css"/>
+</head>
+
+<body>
+
+<h1>Introduction</h1>
+<p>
+    With any sufficiently complex system, the designers and maintainers must make tradeoffs. Apache NiFi is no exception.
+</p>
+
+<p>
+    NiFi is geared toward being run in an environment in which it is free to consume virtually all system resources,
+    especially CPU, disk, and network bandwidth. It is designed in such a way that data is pulled from a source system, optionally filtered, routed,
+    and transformed, before ultimately being published to zero or more destinations. Moreover, the architecture lends itself best to situations in
+    which the source and destinations of a particular piece of data (FlowFile) are themselves loosely coupled.
+</p>
+
+<p>
+    As such, NiFi stores all FlowFile content on disk in order to be resilient across restarts. It provides backpressure so that data consumers
+    avoid overwhelming the system if the data publishers/producers are not able to keep up for some amount of time. It provides the ability to
+    assign more resources to individual parts of a dataflow (via the Concurrent Tasks configuration).
+</p>
+
+<p>
+    All of these design decisions have served NiFi well, making it a leading platform for data integration. However, there are some use cases
+    which lend themselves better to a slightly different architecture than what is provided by traditional NiFi.
+</p>
+
+<p>
+    For example, some use cases are better served by an architecture in which data is not persisted across restarts. Where, instead of storing the
+    data that has been received, the user knows that the data source is both persistent and replayable. In such a situation, it might make more sense
+    to avoid persisting the data and instead source the data anew after restart. This would provide an advantage in that data could potentially be stored
+    in memory instead of on disk, which can provide better performance. Additionally, it provides the ability to move the processing from one machine
+    to another machine without needing to worry about data loss.
+</p>
+
+<p>
+    Stateless NiFi provides a different Runtime Engine than traditional NiFi. It is a single-threaded runtime engine, in which data is not persisted across
+    restarts. Additionally, the data that is sourced can be processed through the entire chain of processors in the dataflow before it is ever even
+    acknowledged from the source. The README document for NiFi Stateless provides far more context as to the differences between traditional NiFi
+    and Stateless NiFi, and you are encouraged to read through it in order to gain a better understanding of the different tradeoffs that were made for the
+    Stateless architecture.
+</p>
+
+<p>
+    Both the traditional NiFi Runtime Engine and the Stateless NiFi Runtime Engine have their strengths and weaknesses. The ideal situation would be one
+    in which users had the ability to easily choose which parts of their dataflow run Stateless and which parts run in the traditional NiFi Runtime Engine.
+</p>
+
+<p>
+    The ExecuteStateless Processor makes this possible.
+</p>
+
+
+
+
+<h1>Configuration</h1>
+
+<p>
+    In order to use the ExecuteStateless Processor, the most important configuration element is the flow definition. That is, where to find that dataflow
+    that is to be run using the Stateless Engine.
+</p>
+
+<h2>Flow Definition</h2>
+<p>
+    The Processor allows the dataflow to be stored in a local file (i.e., a file local to the NiFi server),
+    a URL that is accessible from the NiFi server, or in a NiFi Registry. Once the flow has been fetched, it is cached in the configured <code>Working Directory</code>
+    for resiliency purposes. If NiFi or the ExecuteStateless Processor is stopped and restarted, we do not want to add a single point of failure by relying
+    on some external service to be available. As a result, when the Processor is started, it will first attempt to fetch the flow from the configured location.
+    If unable to do so, it will load the dataflow from the cache, if it is available.
+</p>
+
+
+<h2>Ports</h2>
+<p>
+    Depending on the dataflow that is to be run, it may obtain its data from some external source, such as a JMS Broker via the ConsumeJMS processor.
+    Or, it may take in data from another point in the NiFi flow. In order to do this, the dataflow must be created with an Input Port
+    at the root level of the dataflow. The ExecuteStateless processor is then able to be configured with an incoming connection. When the processor is
+    triggered to run, it will take one FlowFile from the incoming connection and enqueue it into the stateless dataflow for the configured Port.
+    If the Processor is configured to have an incoming Connection, the Input Port property must also be configured, unless there is exactly one
+    Input Port in the dataflow.
+</p>
+
+<p>
+    Similarly, after completing its processing, the stateless flow may route data to one or more Output Ports. Data routed to these Output Ports
+    will then be transferred to the <code>output</code> relationship of the ExecuteStateless Processor. Any FlowFile routed to the <code>output</code>
+    relationship will also have an attribute added to it named "output.port.name" which can be used to route the data if necessary.
+</p>
+
+<p>
+    It is a common practice, however, to have ports that use a naming convention such as "success" and "failure." It may not make sense to have the
+    dataflow take in a FlowFile into its Input Port, perform some processing, and route 1 FlowFile to "success" and route another to "failure" and then
+    to have all of the FlowFile transferred to the <code>output</code> relationship together. We are likely to want to consider the processing of the
+    incoming FlowFile to be a failure if any FlowFile makes its way to the "failure" port. In such a case, we want nothing to go to the "output" relationship,
+    and we want the incoming FlowFile instead to be routed to the "failure" relationship of ExecuteStateless. We can accomplish this by simplify providing
+    a comma-separated list of the Output Ports in the dataflow that should be considered a "failure."
+</p>
+
+
+
+<h1>Success and Failure</h1>
+
+<p>
+    If the ExecuteStateless Processor is configured with an incoming connection, the data will be transferred to one of three relationships:
+    "original," "failure," or "timeout."
+</p>
+
+<p>
+    When the dataflow is triggered, it will have up to the configured amount of time to complete its processing. This time period is configured via
+    the "Dataflow Timeout" property. If the dataflow has not completed in the allotted amount of time, the dataflow is canceled, and the input FlowFile
+    is routed to the "timeout" relationship.
+</p>
+
+<p>
+    If any Processor within the dataflow throws an Exception that it does not handle, the dataflow is considered a failure, and the input FlowFile
+    will be routed to the "failure" relationship.
+</p>
+
+<p>
+    Additionally, if any FlowFile is routed to one of the Ports whose name is defined in the "Failure Ports" property of ExecuteStateless, the dataflow
+    is considered a failure. In this case, an attribute named "failure.port.name" is added to the FlowFile, as there may be multiple ports that are
+    considered failures, and this can be used in order to differentiate between them.
+</p>
+
+<p>
+    Otherwise, the incoming FlowFile will be routed to the "original" relationship, and any FlowFiles routed to any Output Port of the dataflow will be
+    transferred to the "output" relationship of the ExecuteStateless Processor. All FlowFiles transferred to the "output" relationship will also have an
+    attribute named "output.port.name."
+</p>
+
+
+<h1>Designing Flows for Stateless</h1>
+
+<p>
+    When designing a flow to use with Stateless, it is important to consider how the flow might want to receive its data and what it might want
+    to do with the data once it is processed. One option is for the flow to fully encapsulate the source of data and all destinations. For example,
+    it might have a ConsumeKafkaRecord processor, perform some processing, and then publish to another topic via PublishKafkaRecord.
+</p>
+
+<p>
+    Another option would be to build a flow that source data from some external source, possibly perform some processing, but not define where the destination
+    of the data. For example, the flow might consist of a ConsumeKafkaRecord_2_6 processor and perform some filtering and transformation, but stop short
+    of publishing the data anywhere. Instead, it can transfer the data to an Output Port, which could then be used by ExecuteStateless to bring that
+    data into the NiFi dataflow.
+</p>
+
+<p>
+    Similarly, a dataflow may not define where it receives its input from, and instead just use an Input Port, so that any dataflow can be built to source
+    data, and then deliver it to this dataflow, which is responsible for preparing and delivering the data.
+</p>
+
+<p>
+    Finally, the dataflow may define neither the source nor the destination of the data. Instead, the dataflow will be built to use an
+    Input Port, it will perform some filtering/routing/transformation, and finally provide its processing results to an Output Port.
+</p>
+
+
+<h2>Input Ports</h2>
+<p>
+    When designing a Stateless dataflow, it is recommended to use zero or one Input Port. It is possible, however, to define multiple Input Ports.
+    In this case, ExecuteStateless Processor needs to be configured by setting the Input Port property in order to dictate which of those Input Ports
+    the incoming data should be transferred to. Note that the property expects the NAME of the Port and not the identifier. It is also important to note
+    that the name is case sensitive.
+</p>
+
+
+<h2>Output Ports</h2>
+<p>
+    While it is recommended not to use more than one Input Port, it often makes sense to make use of multiple Output Ports. For example, consider a dataflow
+    that takes in CSV data representing information about book sales. The dataflow then partitions the data into "large sales" and "small sales," performs
+    some enrichment, and converts the results into JSON. This dataflow might have four different output ports: "Input CSV," "Large Sales," "Small Sales,"
+    and "Failure."
+</p>
+
+
+
+<h2>Parameters</h2>
+<p>
+    When we build a dataflow, it is often important that we not run the flow with the exact same configuration in every situation. For example, if we are
+    consuming from Kafka, we may want to parameterize the Kafka Brokers, and the name of the Topic. This is best done by making use of Parameters when
+    building the dataflow.
+</p>
+
+<p>
+    Once some value has been parameterized, though, we must have some way of conveying values for those parameters to the ExecuteStateless Processor.
+    To do this, we use user-defined properties. When configuring the ExecuteStateless Processor, in the Properties tab, we can click the '+' icon in the
+    top-right. This allows us to add a custom property to the Processor. Whatever is used for the name and value of that property will be used as the name
+    and value of a parameter in the flow.
+</p>
+
+<p>
+    For example, if our dataflow references a Parameter named "Kafka Topic" and we want to run our dataflow using a value of "book-sales," then we can add
+    a property to ExecuteStateless with the name "Kafka Topic" and the value "book-sales."
+</p>
+
+<p>
+    It is important to note, however, that often times we need the ability to make use of Sensitive Parameters. For example, we may want to parameterize
+    a "password" property. Because of this, any property that is added to ExecuteStateless is considered a sensitive property.
+</p>
+
+
+
+<h1>Exposing the Dataflow</h1>
+
+<p>
+    Now that we've discussed some considerations for building our dataflow, we must consider how we can expose the dataflow, or make the dataflow available
+    to the ExecuteStateless processor.
+</p>
+
+<p>
+    We have three options for this. Firstly, we can right-click on the Process Group that we want to expose, and choose to add the Process Group to Version Control
+    by adding it to the NiFi Registry. This is the recommended approach.
+</p>
+
+<p>
+    However, we can also right-click on the Process Group and instead choose to "Download flow definition." At this point, we can copy the flow definition
+    JSON file to every node in our cluster. Or, alternatively, we can upload the flow definition to some location that is accessible via a URL from every
+    node in the cluster. For example, we might choose to check the JSON file into a Git repository and provide the URL to that file to the processor.
+</p>
+
+<p>
+    It is worth noting that if we define the location of the dataflow to be some external URL or to live within the NiFi Registry, we don't want to add a
+    dependency on that external service to be available and accessible. As a result, when the dataflow is downloaded, it will be cached in the configured
+    <code>Working Directory</code> and if unable to access the flow at some later time, that cached version will be used.
+</p>
+
+<p>
+    It is also important to note that when using an external URL or NiFi Registry, if the Processor is stopped and started (or NiFi is restarted), it is possible that
+    the dataflow could be different than the last time that it ran. Additionally, it's possible that two nodes in a cluster could potentially be running
+    a different version of the flow if they downloaded the file at different times (or if a different file were copied to the nodes). When using NiFi Registry, this can be
+    avoided by explicitly specifying the version of the flow to run.
+</p>
+
+
+
+<h1>Surfacing NiFi Concepts</h1>
+
+<p>
+    Because this one processor is responsible for internally running an entire dataflow, there are several concepts that must be taken into consideration.
+</p>
+
+<h2>Data Provenance</h2>
+<p>
+    Throughout the course of a dataflow, many different intermediate FlowFiles may be created, destroyed, transformed, delivered, and fetched.
+    While traditional NiFi will emit Provenance events for each of these, it is not currently possible with the ExecuteStateless Processor.
+    Because those intermediate FlowFiles are not available, we cannot surface Provenance Events that are emitted by the dataflow, such as SEND
+    and RECEIVE events, because the FlowFiles that were sent and received are not available.
+</p>
+
+<p>
+    Any FlowFile that is transferred to the "output" relationship will be shown as a CREATE Provenance event if there is no input FlowFile.
+    If there is an input FlowFile, those FlowFiles will be shown as FORK events, the child having forked from the incoming FlowFile.
+</p>
+
+
+<h2>Counters</h2>
+
+<p>
+    If any component within the stateless dataflow adjusts a counter, the counters that are adjusted are surfaced as counters for the ExecuteStateless
+    Processor. Consider a dataflow that takes in a single FlowFile and partitions it into two FlowFiles, which are then sent to different Output Ports.
+    Also consider that the partitioning is performed by a PartitionRecord processor with name PartitionData and ID 167ed9c3-a954-3dba-b6fd-c2e1a4572287.
+    Then, we may see a counter for the ExecuteStateless processor with a name "Records Processed - PartitionData (167ed9c3-a954-3dba-b6fd-c2e1a4572287)."
+    This is because the PartitionRecord Processor updates a counter with the name "Records Processed." The additional name and ID of the Processor
+    are added in order to give context.
+</p>
+
+<p>
+    The above mentioned counter, though, will only be incremented for successful invocations of the dataflow. It may be helpful to understand how many
+    times the counter was updated for failed attempts, also. However, we don't want to combine the counters for successful and failed attempts, because
+    that can lead to confusion. Therefore, if the PartitionRecord processor is successful and updates the counter, but the dataflow fails (for example,
+    a FlowFile is then routed to a Failure Port), the ExecuteStateless processor will now have two counters:
+    "Records Processed - PartitionData (167ed9c3-a954-3dba-b6fd-c2e1a4572287)" and
+    "Records Processed - PartitionData (167ed9c3-a954-3dba-b6fd-c2e1a4572287) (Failed attempts)."
+</p>
+
+
+
+<h2>Bulletins</h2>
+<p>
+    We must also consider how bulletins from Processors within the stateless flow get surfaced to the ExecuteStateless processor. This can be helpful
+    for indicating that some concerning behavior is taking place. Any bulletin that is created while running the stateless flow that is at a level of
+    WARNING or ERROR will result in a bulletin being created by the ExecuteStateless Processor (assuming that the ExecuteStateless Processor's Bulletin
+    Level is set sufficiently high in its Settings tab).
+</p>
+
+
+<h2>Yielding</h2>
+<p>
+    Similarly, if any Processor in the Stateless flow chooses to yield, the ExecuteStateless processor will yield. This is important if there are source
+    or destination systems that the Stateless flow is unable to communicate with or that have no more data to offer, as this allows us to avoid constantly
+    interacting with that external service, which could add significant load to it.
+</p>
+
+
+
+
+<h1>Performance Considerations</h1>
+<p>
+    There are a few different performance-related considerations to take into effect when configuring the ExecuteStateless Processor.
+</p>
+
+
+<h2>Content Storage Strategy</h2>
+<p>
+    One of the most impactful configuration options for the Processor is the configuration of the "Content Storage Strategy" property. For performance
+    reasons, the processor can be configured to hold all FlowFiles in memory. This includes incoming FlowFiles, as well as intermediate and output FlowFiles.
+    This can be a significant performance improvement but comes with a significant risk. The content is stored on NiFi's heap. This is the same heap that is shared
+    by all other ExecuteStateless flows and by NiFi's processors and the NiFi process itself. If the data is very large, it can quickly exhaust the heap, resulting
+    in Out Of Memory Errors in NiFi. These, in turn, can result in poor performance, as well as instability of the NiFi process itself. For this reason, it is not
+    recommended to use the "Store Content on Heap" option unless it is known that all FlowFiles will be small (less than a few MB). And in order to help safeguard
+    against the case that the Processor receives an unexpectedly large FlowFile, the "Max Input FlowFile Size" property must be configured when storing data on the heap.
+</p>
+
+<p>
+    Alternatively, and by default, the "Content Storage Strategy" can be configured to store FlowFile content on disk. When this option is used, the content of all
+    FlowFiles is stored in the configured <code>Working Directory</code>. It is important to note, however, that this data is not meant to be persisted across restarts.
+    Instead, this simply provides the Stateless Engine with a way to avoid loading everything into memory. Upon restart, the data will be deleted instead of allowing
+    FlowFiles to resume from where they left off.
+</p>
+
+
+<h2>Concurrent Tasks</h2>
+<p>
+    As noted before, the Stateless Engine is single-threaded. However, the processor does allow the user to configure more than one concurrent task. In this situation,
+    each thread/concurrent task will run its own instance of the dataflow. This functions in much the same way as if a single thread were run on each of many different computers.
+    Any internal state that is stored by the processor, such as the creation of a client for interacting with another service, is not shared. Additionally, if any Processors are
+    configured to run on Primary Node only, they will run for each instance.
+</p>
+
+
+<h2>Run Duration</h2>
+<p>
+    This Processor supports the configuration of NiFi's Run Duration in the Scheduling tab. If the Processor is expected to process many small FlowFiles, it is recommended to configure
+    this option so that the Processor has a Run Duration of "25 ms." Typically, adjusting the value beyond that offers little benefit, but adjusting from "0 ms" to "25 ms" can
+    provide a very significant performance improvement for many dataflows, at the cost of potentially introducing up to 25 milliseconds worth of additional latency.
+</p>
+
+</body>
+</html>
diff --git a/nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/test/java/org/apache/nifi/processors/stateless/retrieval/TestCachingDataflowProvider.java b/nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/test/java/org/apache/nifi/processors/stateless/retrieval/TestCachingDataflowProvider.java
new file mode 100644
index 0000000..8791994
--- /dev/null
+++ b/nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/test/java/org/apache/nifi/processors/stateless/retrieval/TestCachingDataflowProvider.java
@@ -0,0 +1,130 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.processors.stateless.retrieval;
+
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.flow.VersionedProcessGroup;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processors.stateless.ExecuteStateless;
+import org.apache.nifi.registry.flow.VersionedFlowSnapshot;
+import org.junit.Test;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import java.io.File;
+import java.io.IOException;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertThrows;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+public class TestCachingDataflowProvider {
+
+    @Test
+    public void testDelegatesEvenAfterFetch() throws IOException {
+        final VersionedFlowSnapshot snapshot = new VersionedFlowSnapshot();
+
+        final DataflowProvider mockedRetrieval = mock(DataflowProvider.class);
+        when(mockedRetrieval.retrieveDataflowContents(any(ProcessContext.class))).thenReturn(snapshot);
+
+        final CachingDataflowProvider retrieval = new CachingDataflowProvider("1234", mock(ComponentLog.class), mockedRetrieval);
+
+        final ProcessContext context = mock(ProcessContext.class);
+        final PropertyValue propertyValue = mock(PropertyValue.class);
+        when(propertyValue.getValue()).thenReturn("target/testFetchFirst/cache");
+        when(context.getProperty(ExecuteStateless.WORKING_DIRECTORY)).thenReturn(propertyValue);
+
+        final File cacheFile = retrieval.getFlowCacheFile(context, "1234");
+
+        for (int i=0; i < 2; i++) {
+            retrieval.retrieveDataflowContents(context);
+            assertTrue(cacheFile.exists());
+        }
+
+        verify(mockedRetrieval, times(2)).retrieveDataflowContents(context);
+    }
+
+    @Test
+    public void testThrowsIfDelegateThrowsAndNoCache() throws IOException {
+        final DataflowProvider mockedRetrieval = mock(DataflowProvider.class);
+        when(mockedRetrieval.retrieveDataflowContents(any(ProcessContext.class))).thenThrow(new IOException("Intentional exception for testing purposes"));
+
+        final CachingDataflowProvider retrieval = new CachingDataflowProvider("1234", mock(ComponentLog.class), mockedRetrieval);
+
+        final ProcessContext context = mock(ProcessContext.class);
+        final PropertyValue propertyValue = mock(PropertyValue.class);
+        when(propertyValue.getValue()).thenReturn("target/testThrowsIfDelegateThrowsAndNoCache/cache");
+        when(context.getProperty(ExecuteStateless.WORKING_DIRECTORY)).thenReturn(propertyValue);
+
+        final File cacheFile = retrieval.getFlowCacheFile(context, "1234");
+
+        assertThrows(IOException.class, () -> retrieval.retrieveDataflowContents(context));
+        assertFalse(cacheFile.exists());
+    }
+
+    @Test
+    public void testFetchesCachedFlowOnException() throws IOException {
+        final VersionedProcessGroup group = new VersionedProcessGroup();
+        group.setName("Testable Group");
+        final VersionedFlowSnapshot snapshot = new VersionedFlowSnapshot();
+        snapshot.setFlowContents(group);
+
+        final DataflowProvider mockedRetrieval = mock(DataflowProvider.class);
+        doAnswer(new Answer<VersionedFlowSnapshot>() {
+            private int count;
+
+            @Override
+            public VersionedFlowSnapshot answer(final InvocationOnMock invocation) throws Throwable {
+                if (count++ == 0) {
+                    return snapshot;
+                }
+
+                throw new IOException("Intentional failure for testing purposes");
+            }
+        }).when(mockedRetrieval).retrieveDataflowContents(any(ProcessContext.class));
+
+        final CachingDataflowProvider retrieval = new CachingDataflowProvider("1234", mock(ComponentLog.class), mockedRetrieval);
+
+        final ProcessContext context = mock(ProcessContext.class);
+        final PropertyValue propertyValue = mock(PropertyValue.class);
+        when(propertyValue.getValue()).thenReturn("target/testFetchesCachedFlowOnException/cache");
+        when(context.getProperty(ExecuteStateless.WORKING_DIRECTORY)).thenReturn(propertyValue);
+
+        final File cacheFile = retrieval.getFlowCacheFile(context, "1234");
+
+        VersionedFlowSnapshot retrieved = null;
+        for (int i=0; i < 2; i++) {
+            retrieved = retrieval.retrieveDataflowContents(context);
+            assertNotNull(retrieved);
+
+            assertTrue(cacheFile.exists());
+        }
+
+        verify(mockedRetrieval, times(2)).retrieveDataflowContents(context);
+        assertEquals(group.getName(), retrieved.getFlowContents().getName());
+    }
+}
diff --git a/nifi-nar-bundles/nifi-stateless-processor-bundle/pom.xml b/nifi-nar-bundles/nifi-stateless-processor-bundle/pom.xml
new file mode 100644
index 0000000..01f3793
--- /dev/null
+++ b/nifi-nar-bundles/nifi-stateless-processor-bundle/pom.xml
@@ -0,0 +1,34 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements. See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License. You may obtain a copy of the License at
+  http://www.apache.org/licenses/LICENSE-2.0
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <artifactId>nifi-nar-bundles</artifactId>
+        <groupId>org.apache.nifi</groupId>
+        <version>1.15.0-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+
+    <artifactId>nifi-stateless-processor-bundle</artifactId>
+    <packaging>pom</packaging>
+    <modules>
+        <module>nifi-stateless-processor</module>
+        <module>nifi-stateless-processor-nar</module>
+        <module>nifi-stateless-processor-tests</module>
+    </modules>
+
+</project>
\ No newline at end of file
diff --git a/nifi-nar-bundles/pom.xml b/nifi-nar-bundles/pom.xml
index cce187d..a3ea9a7 100755
--- a/nifi-nar-bundles/pom.xml
+++ b/nifi-nar-bundles/pom.xml
@@ -107,6 +107,7 @@
 	    <module>nifi-asn1-bundle</module>
         <module>nifi-pgp-bundle</module>
         <module>nifi-hashicorp-vault-bundle</module>
+        <module>nifi-stateless-processor-bundle</module>
     </modules>
 
     <build>
diff --git a/nifi-registry/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/link/LinkAdapter.java b/nifi-registry/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/link/LinkAdapter.java
index 76bd708..3dba899 100644
--- a/nifi-registry/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/link/LinkAdapter.java
+++ b/nifi-registry/nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/link/LinkAdapter.java
@@ -37,6 +37,10 @@ public class LinkAdapter extends XmlAdapter<JaxbLink, Link> {
             return null;
         }
 
+        if (v.getUri() == null) {
+            return null;
+        }
+
         Link.Builder lb = Link.fromUri(v.getUri());
         if (v.getParams() != null) {
             for (Map.Entry<String,String> e : v.getParams().entrySet()) {
diff --git a/nifi-stateless/nifi-stateless-api/src/main/java/org/apache/nifi/stateless/engine/StatelessEngineConfiguration.java b/nifi-stateless/nifi-stateless-api/src/main/java/org/apache/nifi/stateless/engine/StatelessEngineConfiguration.java
index 60b5ddb..8f27c66 100644
--- a/nifi-stateless/nifi-stateless-api/src/main/java/org/apache/nifi/stateless/engine/StatelessEngineConfiguration.java
+++ b/nifi-stateless/nifi-stateless-api/src/main/java/org/apache/nifi/stateless/engine/StatelessEngineConfiguration.java
@@ -43,4 +43,8 @@ public interface StatelessEngineConfiguration {
     String getSensitivePropsKey();
 
     List<ExtensionClientDefinition> getExtensionClients();
+
+    default boolean isLogExtensionDiscovery() {
+        return true;
+    }
 }
diff --git a/nifi-stateless/nifi-stateless-api/src/main/java/org/apache/nifi/stateless/flow/FailurePortEncounteredException.java b/nifi-stateless/nifi-stateless-api/src/main/java/org/apache/nifi/stateless/flow/DataflowTriggerContext.java
similarity index 60%
copy from nifi-stateless/nifi-stateless-api/src/main/java/org/apache/nifi/stateless/flow/FailurePortEncounteredException.java
copy to nifi-stateless/nifi-stateless-api/src/main/java/org/apache/nifi/stateless/flow/DataflowTriggerContext.java
index fe18907..d2ee77d 100644
--- a/nifi-stateless/nifi-stateless-api/src/main/java/org/apache/nifi/stateless/flow/FailurePortEncounteredException.java
+++ b/nifi-stateless/nifi-stateless-api/src/main/java/org/apache/nifi/stateless/flow/DataflowTriggerContext.java
@@ -17,10 +17,20 @@
 
 package org.apache.nifi.stateless.flow;
 
-public class FailurePortEncounteredException extends RuntimeException {
-
-    public FailurePortEncounteredException(String message) {
-        super(message);
-    }
+public interface DataflowTriggerContext {
+    /**
+     * Provides a mechanism by which the triggering class can abort a dataflow
+     * @return <code>true</code> if the dataflow should be aborted, <code>false</code> otherwise
+     */
+    boolean isAbort();
 
+    /**
+     * The implicit context that will be used if no other context is provided when triggering a dataflow
+     */
+    DataflowTriggerContext IMPLICIT_CONTEXT = new DataflowTriggerContext() {
+        @Override
+        public boolean isAbort() {
+            return false;
+        }
+    };
 }
diff --git a/nifi-stateless/nifi-stateless-api/src/main/java/org/apache/nifi/stateless/flow/FailurePortEncounteredException.java b/nifi-stateless/nifi-stateless-api/src/main/java/org/apache/nifi/stateless/flow/FailurePortEncounteredException.java
index fe18907..6006d55 100644
--- a/nifi-stateless/nifi-stateless-api/src/main/java/org/apache/nifi/stateless/flow/FailurePortEncounteredException.java
+++ b/nifi-stateless/nifi-stateless-api/src/main/java/org/apache/nifi/stateless/flow/FailurePortEncounteredException.java
@@ -18,9 +18,15 @@
 package org.apache.nifi.stateless.flow;
 
 public class FailurePortEncounteredException extends RuntimeException {
+    private final String portName;
 
-    public FailurePortEncounteredException(String message) {
+    public FailurePortEncounteredException(final String message, final String portName) {
         super(message);
+        this.portName = portName;
+    }
+
+    public String getPortName() {
+        return portName;
     }
 
 }
diff --git a/nifi-stateless/nifi-stateless-api/src/main/java/org/apache/nifi/stateless/flow/StatelessDataflow.java b/nifi-stateless/nifi-stateless-api/src/main/java/org/apache/nifi/stateless/flow/StatelessDataflow.java
index be6e62c..dbd44cc 100644
--- a/nifi-stateless/nifi-stateless-api/src/main/java/org/apache/nifi/stateless/flow/StatelessDataflow.java
+++ b/nifi-stateless/nifi-stateless-api/src/main/java/org/apache/nifi/stateless/flow/StatelessDataflow.java
@@ -19,18 +19,32 @@ package org.apache.nifi.stateless.flow;
 
 import org.apache.nifi.components.state.Scope;
 import org.apache.nifi.controller.queue.QueueSize;
+import org.apache.nifi.reporting.BulletinRepository;
 
+import java.io.InputStream;
 import java.util.Map;
 import java.util.Set;
 
 public interface StatelessDataflow {
     /**
+     * Triggers the dataflow to run, returning a DataflowTrigger that can be used to wait for the result. Uses the {@link DataflowTriggerContext#IMPLICIT_CONTEXT}.
+     * @return a DataflowTrigger that can be used to wait for the result
+     *
+     * @throws IllegalStateException if called before {@link #initialize()} is called.
+     */
+    default DataflowTrigger trigger() {
+        return trigger(DataflowTriggerContext.IMPLICIT_CONTEXT);
+    }
+
+    /**
      * Triggers the dataflow to run, returning a DataflowTrigger that can be used to wait for the result
+     *
+     * @param triggerContext the trigger context to use
      * @return a DataflowTrigger that can be used to wait for the result
      *
      * @throws IllegalStateException if called before {@link #initialize()} is called.
      */
-    DataflowTrigger trigger();
+    DataflowTrigger trigger(DataflowTriggerContext triggerContext);
 
     /**
      * <p>
@@ -61,6 +75,8 @@ public interface StatelessDataflow {
 
     QueueSize enqueue(byte[] flowFileContents, Map<String, String> attributes, String portName);
 
+    QueueSize enqueue(InputStream flowFileContents, Map<String, String> attributes, String portName);
+
     boolean isFlowFileQueued();
 
     void purge();
@@ -72,4 +88,10 @@ public interface StatelessDataflow {
     boolean isSourcePrimaryNodeOnly();
 
     long getSourceYieldExpiration();
+
+    void resetCounters();
+
+    Map<String, Long> getCounters(boolean includeGlobalContext);
+
+    BulletinRepository getBulletinRepository();
 }
diff --git a/nifi-stateless/nifi-stateless-api/src/main/java/org/apache/nifi/stateless/flow/TriggerResult.java b/nifi-stateless/nifi-stateless-api/src/main/java/org/apache/nifi/stateless/flow/TriggerResult.java
index fc8bba1..3d4de5f 100644
--- a/nifi-stateless/nifi-stateless-api/src/main/java/org/apache/nifi/stateless/flow/TriggerResult.java
+++ b/nifi-stateless/nifi-stateless-api/src/main/java/org/apache/nifi/stateless/flow/TriggerResult.java
@@ -18,6 +18,7 @@
 package org.apache.nifi.stateless.flow;
 
 import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.provenance.ProvenanceEventRecord;
 
 import java.io.IOException;
 import java.io.InputStream;
@@ -82,4 +83,10 @@ public interface TriggerResult {
      * @param cause the cause for aborting the dataflow, or <code>null</code> if no cause is to be specified
      */
     void abort(Throwable cause);
+
+    /**
+     * Returns all Provenance Events that were created during this invocation of the dataflow
+     * @return the list of Provenance events
+     */
+    List<ProvenanceEventRecord> getProvenanceEvents() throws IOException;
 }
diff --git a/nifi-stateless/nifi-stateless-bootstrap/src/main/java/org/apache/nifi/stateless/bootstrap/StatelessBootstrap.java b/nifi-stateless/nifi-stateless-bootstrap/src/main/java/org/apache/nifi/stateless/bootstrap/StatelessBootstrap.java
index 426b65f..d99cea5 100644
--- a/nifi-stateless/nifi-stateless-bootstrap/src/main/java/org/apache/nifi/stateless/bootstrap/StatelessBootstrap.java
+++ b/nifi-stateless/nifi-stateless-bootstrap/src/main/java/org/apache/nifi/stateless/bootstrap/StatelessBootstrap.java
@@ -49,9 +49,6 @@ import java.util.regex.Pattern;
 public class StatelessBootstrap {
     private static final Logger logger = LoggerFactory.getLogger(StatelessBootstrap.class);
     private static final Pattern STATELESS_NAR_PATTERN = Pattern.compile("nifi-stateless-nar-.*\\.nar-unpacked");
-    private static final String NIFI_GROUP = "org.apache.nifi";
-    private static final String NIFI_STATELESS_ARTIFACT_ID = "nifi-stateless-nar";
-    private static final String NIFI_JETTY_ARTIFACT_ID = "nifi-jetty-bundle";
     private final ClassLoader statelessClassLoader;
     private final StatelessEngineConfiguration engineConfiguration;
 
@@ -88,15 +85,16 @@ public class StatelessBootstrap {
     public static StatelessBootstrap bootstrap(final StatelessEngineConfiguration engineConfiguration, final ClassLoader rootClassLoader) throws IOException {
         final File narDirectory = engineConfiguration.getNarDirectory();
         final File workingDirectory = engineConfiguration.getWorkingDirectory();
+        final File narExpansionDirectory = new File(workingDirectory, "nar");
 
         // Ensure working directory exists, creating it if necessary
-        if (!workingDirectory.exists() && !workingDirectory.mkdirs()) {
-            throw new IOException("Working Directory " + workingDirectory + " does not exist and could not be created");
+        if (!narExpansionDirectory.exists() && !narExpansionDirectory.mkdirs()) {
+            throw new IOException("Working Directory " + narExpansionDirectory + " does not exist and could not be created");
         }
 
         final Bundle systemBundle = SystemBundle.create(narDirectory.getAbsolutePath(), ClassLoader.getSystemClassLoader());
-        final File frameworkWorkingDir = new File(workingDirectory, "nifi-framework");
-        final File extensionsWorkingDir = new File(workingDirectory, "extensions");
+        final File frameworkWorkingDir = new File(narExpansionDirectory, "framework");
+        final File extensionsWorkingDir = new File(narExpansionDirectory, "extensions");
         final List<Path> narDirectories = Collections.singletonList(narDirectory.toPath());
 
         // Unpack NARs
@@ -131,16 +129,6 @@ public class StatelessBootstrap {
         return new StatelessBootstrap(statelessClassLoader, engineConfiguration);
     }
 
-    private static boolean isRequiredForBootstrap(final BundleCoordinate coordinate) {
-        final String group = coordinate.getGroup();
-        if (!NIFI_GROUP.equals(group)) {
-            return false;
-        }
-
-        final String artifactId = coordinate.getId();
-        return NIFI_JETTY_ARTIFACT_ID.equals(artifactId) || NIFI_STATELESS_ARTIFACT_ID.equals(artifactId);
-    }
-
     private static File locateStatelessNarWorkingDirectory(final File workingDirectory) throws IOException {
         final File[] files = workingDirectory.listFiles();
         if (files == null) {
diff --git a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/bootstrap/ExtensionDiscovery.java b/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/bootstrap/ExtensionDiscovery.java
index 8430978..837bbb5 100644
--- a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/bootstrap/ExtensionDiscovery.java
+++ b/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/bootstrap/ExtensionDiscovery.java
@@ -33,11 +33,12 @@ import java.util.concurrent.TimeUnit;
 public class ExtensionDiscovery {
     private static final Logger logger = LoggerFactory.getLogger(ExtensionDiscovery.class);
 
-    public static ExtensionDiscoveringManager discover(final File narWorkingDirectory, final ClassLoader systemClassLoader, final NarClassLoaders narClassLoaders) throws IOException {
+    public static ExtensionDiscoveringManager discover(final File narWorkingDirectory, final ClassLoader systemClassLoader, final NarClassLoaders narClassLoaders,
+                                                       final boolean logExtensionDiscovery) throws IOException {
         logger.info("Initializing NAR ClassLoaders");
 
         try {
-            narClassLoaders.init(systemClassLoader, null, narWorkingDirectory);
+            narClassLoaders.init(systemClassLoader, null, narWorkingDirectory, logExtensionDiscovery);
         } catch (final ClassNotFoundException cnfe) {
             throw new IOException("Could not initialize Class Loaders", cnfe);
         }
@@ -46,7 +47,7 @@ public class ExtensionDiscovery {
 
         final long discoveryStart = System.nanoTime();
         final StandardExtensionDiscoveringManager extensionManager = new StandardExtensionDiscoveringManager(Collections.singleton(ParameterValueProvider.class));
-        extensionManager.discoverExtensions(narBundles);
+        extensionManager.discoverExtensions(narBundles, logExtensionDiscovery);
 
         final long discoveryMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - discoveryStart);
         logger.info("Successfully discovered extensions in {} milliseconds", discoveryMillis);
diff --git a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/engine/ExecutionProgress.java b/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/engine/ExecutionProgress.java
index 2c475be..f3ec125 100644
--- a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/engine/ExecutionProgress.java
+++ b/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/engine/ExecutionProgress.java
@@ -51,6 +51,12 @@ public interface ExecutionProgress {
      */
     void notifyExecutionFailed(Throwable cause);
 
+    /**
+     * Indicates whether or not the port with the given name is considered a Failure Port
+     * @param portName the name of the port
+     * @return <code>true</code> if the port is a failure port, <code>false</code> otherwise
+     */
+    boolean isFailurePort(String portName);
 
     enum CompletionAction {
         COMPLETE,
diff --git a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/engine/StandardExecutionProgress.java b/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/engine/StandardExecutionProgress.java
index 9b8f0a7..12427fb 100644
--- a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/engine/StandardExecutionProgress.java
+++ b/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/engine/StandardExecutionProgress.java
@@ -27,9 +27,15 @@ import org.apache.nifi.controller.repository.claim.ContentClaim;
 import org.apache.nifi.controller.repository.io.LimitedInputStream;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.groups.ProcessGroup;
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+import org.apache.nifi.provenance.ProvenanceEventRepository;
+import org.apache.nifi.stateless.flow.CanceledTriggerResult;
+import org.apache.nifi.stateless.flow.DataflowTriggerContext;
+import org.apache.nifi.stateless.flow.ExceptionalTriggerResult;
 import org.apache.nifi.stateless.flow.FailurePortEncounteredException;
 import org.apache.nifi.stateless.flow.TriggerResult;
 import org.apache.nifi.stateless.queue.DrainableFlowFileQueue;
+import org.apache.nifi.stateless.repository.RepositoryContextFactory;
 import org.apache.nifi.stateless.session.AsynchronousCommitTracker;
 import org.apache.nifi.stream.io.StreamUtils;
 
@@ -49,32 +55,53 @@ public class StandardExecutionProgress implements ExecutionProgress {
     private final ProcessGroup rootGroup;
     private final List<FlowFileQueue> internalFlowFileQueues;
     private final ContentRepository contentRepository;
+    private final ProvenanceEventRepository provenanceRepository;
     private final BlockingQueue<TriggerResult> resultQueue;
     private final Set<String> failurePortNames;
     private final AsynchronousCommitTracker commitTracker;
     private final StatelessStateManagerProvider stateManagerProvider;
+    private final Long maxProvenanceEventId;
+    private final DataflowTriggerContext triggerContext;
 
     private final BlockingQueue<CompletionAction> completionActionQueue;
     private volatile boolean canceled = false;
     private volatile CompletionAction completionAction = null;
 
     public StandardExecutionProgress(final ProcessGroup rootGroup, final List<FlowFileQueue> internalFlowFileQueues, final BlockingQueue<TriggerResult> resultQueue,
-                                     final ContentRepository contentRepository, final Set<String> failurePortNames, final AsynchronousCommitTracker commitTracker,
-                                     final StatelessStateManagerProvider stateManagerProvider) {
+                                     final RepositoryContextFactory repositoryContextFactory, final Set<String> failurePortNames, final AsynchronousCommitTracker commitTracker,
+                                     final StatelessStateManagerProvider stateManagerProvider, final DataflowTriggerContext triggerContext) {
         this.rootGroup = rootGroup;
         this.internalFlowFileQueues = internalFlowFileQueues;
         this.resultQueue = resultQueue;
-        this.contentRepository = contentRepository;
+        this.contentRepository = repositoryContextFactory.getContentRepository();
+        this.provenanceRepository = repositoryContextFactory.getProvenanceRepository();
         this.failurePortNames = failurePortNames;
         this.commitTracker = commitTracker;
         this.stateManagerProvider = stateManagerProvider;
+        this.maxProvenanceEventId = provenanceRepository.getMaxEventId();
+        this.triggerContext = triggerContext;
 
         completionActionQueue = new LinkedBlockingQueue<>();
     }
 
     @Override
+    public boolean isFailurePort(final String portName) {
+        return failurePortNames.contains(portName);
+    }
+
+    @Override
     public boolean isCanceled() {
-        return canceled;
+        if (canceled) {
+            return true;
+        }
+
+        final boolean aborted = triggerContext.isAbort();
+        if (aborted) {
+            notifyExecutionCanceled();
+            return true;
+        }
+
+        return false;
     }
 
     @Override
@@ -90,7 +117,7 @@ public class StandardExecutionProgress implements ExecutionProgress {
 
     @Override
     public CompletionAction awaitCompletionAction() throws InterruptedException {
-        if (canceled) {
+        if (isCanceled()) {
             return CompletionAction.CANCEL;
         }
 
@@ -114,7 +141,7 @@ public class StandardExecutionProgress implements ExecutionProgress {
         for (final String failurePortName : failurePortNames) {
             final List<FlowFile> flowFilesForPort = outputFlowFiles.get(failurePortName);
             if (flowFilesForPort != null && !flowFilesForPort.isEmpty()) {
-                throw new FailurePortEncounteredException("FlowFile was transferred to Port " + failurePortName + ", which is marked as a Failure Port");
+                throw new FailurePortEncounteredException("FlowFile was transferred to Port " + failurePortName + ", which is marked as a Failure Port", failurePortName);
             }
         }
 
@@ -202,6 +229,11 @@ public class StandardExecutionProgress implements ExecutionProgress {
                 abortCause = new DataflowAbortedException("Dataflow was aborted", cause);
                 notifyExecutionFailed(abortCause);
             }
+
+            @Override
+            public List<ProvenanceEventRecord> getProvenanceEvents() throws IOException {
+                return provenanceRepository.getEvents(maxProvenanceEventId == null ? 0 : maxProvenanceEventId + 1, Integer.MAX_VALUE);
+            }
         };
     }
 
@@ -211,6 +243,7 @@ public class StandardExecutionProgress implements ExecutionProgress {
         commitTracker.triggerFailureCallbacks(new RuntimeException("Dataflow Canceled"));
         stateManagerProvider.rollbackUpdates();
         completionActionQueue.offer(CompletionAction.CANCEL);
+        resultQueue.offer(new CanceledTriggerResult());
         contentRepository.purge();
     }
 
@@ -219,6 +252,7 @@ public class StandardExecutionProgress implements ExecutionProgress {
         commitTracker.triggerFailureCallbacks(cause);
         stateManagerProvider.rollbackUpdates();
         completionActionQueue.offer(CompletionAction.CANCEL);
+        resultQueue.offer(new ExceptionalTriggerResult(cause));
         contentRepository.purge();
     }
 
diff --git a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/engine/StandardStatelessEngine.java b/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/engine/StandardStatelessEngine.java
index d9d1510..16488f4 100644
--- a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/engine/StandardStatelessEngine.java
+++ b/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/engine/StandardStatelessEngine.java
@@ -42,7 +42,9 @@ import org.apache.nifi.controller.service.ControllerServiceProvider;
 import org.apache.nifi.encrypt.PropertyEncryptor;
 import org.apache.nifi.engine.FlowEngine;
 import org.apache.nifi.extensions.ExtensionRepository;
+import org.apache.nifi.flow.VersionedProcessGroup;
 import org.apache.nifi.groups.ProcessGroup;
+import org.apache.nifi.logging.LogRepositoryFactory;
 import org.apache.nifi.nar.ExtensionDefinition;
 import org.apache.nifi.nar.ExtensionManager;
 import org.apache.nifi.nar.InstanceClassLoader;
@@ -56,7 +58,6 @@ import org.apache.nifi.registry.VariableRegistry;
 import org.apache.nifi.registry.flow.FlowRegistryClient;
 import org.apache.nifi.registry.flow.VersionedFlow;
 import org.apache.nifi.registry.flow.VersionedFlowSnapshot;
-import org.apache.nifi.flow.VersionedProcessGroup;
 import org.apache.nifi.reporting.BulletinRepository;
 import org.apache.nifi.reporting.ReportingTask;
 import org.apache.nifi.scheduling.SchedulingStrategy;
@@ -170,12 +171,12 @@ public class StandardStatelessEngine implements StatelessEngine<VersionedFlowSna
         childGroup.setName("Stateless Flow");
         rootGroup.addProcessGroup(childGroup);
 
+        LogRepositoryFactory.purge();
         childGroup.updateFlow(dataflowDefinition.getFlowSnapshot(), "stateless-component-id-seed", false, true, true);
 
         final ParameterValueProvider parameterValueProvider = createParameterValueProvider(dataflowDefinition);
 
         // Map existing parameter contexts by name
-        final Set<ParameterContext> parameterContexts = flowManager.getParameterContextManager().getParameterContexts();
         final Map<String, ParameterContext> parameterContextMap = flowManager.getParameterContextManager().getParameterContextNameMapping();
 
         // Update Parameters to match those that are provided in the flow configuration, plus those overrides provided
@@ -188,7 +189,7 @@ public class StandardStatelessEngine implements StatelessEngine<VersionedFlowSna
 
         final List<ReportingTaskNode> reportingTaskNodes = createReportingTasks(dataflowDefinition);
         final StandardStatelessFlow dataflow = new StandardStatelessFlow(childGroup, reportingTaskNodes, controllerServiceProvider, processContextFactory,
-            repositoryContextFactory, dataflowDefinition, stateManagerProvider, processScheduler);
+            repositoryContextFactory, dataflowDefinition, stateManagerProvider, processScheduler, bulletinRepository);
 
         final LogComponentStatuses logComponentStatuses = new LogComponentStatuses(flowFileEventRepository, counterRepository, flowManager);
         dataflow.scheduleBackgroundTask(logComponentStatuses, 1, TimeUnit.MINUTES);
@@ -223,6 +224,7 @@ public class StandardStatelessEngine implements StatelessEngine<VersionedFlowSna
         final String providerId = UUID.randomUUID().toString();
         final InstanceClassLoader classLoader = extensionManager.createInstanceClassLoader(providerType, providerId, bundle, Collections.emptySet());
 
+        final ClassLoader initialClassLoader = Thread.currentThread().getContextClassLoader();
         try {
             final Class<?> rawClass = Class.forName(providerType, true, classLoader);
             Thread.currentThread().setContextClassLoader(classLoader);
@@ -243,6 +245,8 @@ public class StandardStatelessEngine implements StatelessEngine<VersionedFlowSna
             return parameterValueProvider;
         } catch (final Exception e) {
             throw new IllegalStateException("Could not create Parameter Value Provider " + definition.getName() + " of type " + definition.getType(), e);
+        } finally {
+            Thread.currentThread().setContextClassLoader(initialClassLoader);
         }
     }
 
diff --git a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/engine/StatelessFlowManager.java b/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/engine/StatelessFlowManager.java
index b132e17..737f737 100644
--- a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/engine/StatelessFlowManager.java
+++ b/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/engine/StatelessFlowManager.java
@@ -51,8 +51,12 @@ import org.apache.nifi.flowfile.FlowFilePrioritizer;
 import org.apache.nifi.groups.ProcessGroup;
 import org.apache.nifi.groups.RemoteProcessGroup;
 import org.apache.nifi.groups.StandardProcessGroup;
+import org.apache.nifi.logging.ControllerServiceLogObserver;
+import org.apache.nifi.logging.LogLevel;
 import org.apache.nifi.logging.LogRepository;
 import org.apache.nifi.logging.LogRepositoryFactory;
+import org.apache.nifi.logging.ProcessorLogObserver;
+import org.apache.nifi.logging.ReportingTaskLogObserver;
 import org.apache.nifi.nar.ExtensionManager;
 import org.apache.nifi.nar.NarCloseable;
 import org.apache.nifi.parameter.ParameterContextManager;
@@ -60,6 +64,7 @@ import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.registry.flow.VersionedFlowSnapshot;
 import org.apache.nifi.registry.variable.MutableVariableRegistry;
 import org.apache.nifi.remote.StandardRemoteProcessGroup;
+import org.apache.nifi.reporting.BulletinRepository;
 import org.apache.nifi.stateless.queue.StatelessFlowFileQueue;
 import org.apache.nifi.util.ReflectionUtils;
 import org.apache.nifi.web.api.dto.FlowSnippetDTO;
@@ -84,14 +89,16 @@ public class StatelessFlowManager extends AbstractFlowManager implements FlowMan
 
     private final StatelessEngine<VersionedFlowSnapshot> statelessEngine;
     private final SSLContext sslContext;
+    private final BulletinRepository bulletinRepository;
 
     public StatelessFlowManager(final FlowFileEventRepository flowFileEventRepository, final ParameterContextManager parameterContextManager,
                                 final StatelessEngine<VersionedFlowSnapshot> statelessEngine, final BooleanSupplier flowInitializedCheck,
-                                final SSLContext sslContext) {
+                                final SSLContext sslContext, final BulletinRepository bulletinRepository) {
         super(flowFileEventRepository, parameterContextManager, statelessEngine.getFlowRegistryClient(), flowInitializedCheck);
 
         this.statelessEngine = statelessEngine;
         this.sslContext = sslContext;
+        this.bulletinRepository = bulletinRepository;
     }
 
     @Override
@@ -163,10 +170,6 @@ public class StatelessFlowManager extends AbstractFlowManager implements FlowMan
             try (final NarCloseable x = NarCloseable.withComponentNarLoader(extensionManager, procNode.getProcessor().getClass(), procNode.getProcessor().getIdentifier())) {
                 ReflectionUtils.invokeMethodsWithAnnotation(OnAdded.class, procNode.getProcessor());
             } catch (final Exception e) {
-                if (registerLogObserver) {
-                    logRepository.removeObserver(StandardProcessorNode.BULLETIN_OBSERVER_ID);
-                }
-
                 throw new ComponentLifeCycleException("Failed to invoke @OnAdded methods of " + procNode.getProcessor(), e);
             }
 
@@ -174,6 +177,11 @@ public class StatelessFlowManager extends AbstractFlowManager implements FlowMan
                 ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnConfigurationRestored.class, procNode.getProcessor());
             }
 
+            LogRepositoryFactory.getRepository(procNode.getIdentifier()).setLogger(procNode.getLogger());
+            if (registerLogObserver) {
+                logRepository.addObserver(StandardProcessorNode.BULLETIN_OBSERVER_ID, procNode.getBulletinLevel(), new ProcessorLogObserver(bulletinRepository, procNode));
+            }
+
             logger.debug("Processor with id {} successfully created", id);
 
             return procNode;
@@ -258,6 +266,8 @@ public class StatelessFlowManager extends AbstractFlowManager implements FlowMan
                 + ", bundle coordinate = " + bundleCoordinate);
         }
 
+        final LogRepository logRepository = LogRepositoryFactory.getRepository(id);
+
         final ReportingTaskNode taskNode;
         try {
             taskNode = new ComponentBuilder()
@@ -291,6 +301,9 @@ public class StatelessFlowManager extends AbstractFlowManager implements FlowMan
 
         if (register) {
             onReportingTaskAdded(taskNode);
+
+            // Register log observer to provide bulletins when reporting task logs anything at WARN level or above
+            logRepository.addObserver(StandardProcessorNode.BULLETIN_OBSERVER_ID, LogLevel.WARN, new ReportingTaskLogObserver(bulletinRepository, taskNode));
         }
 
         return taskNode;
@@ -314,7 +327,10 @@ public class StatelessFlowManager extends AbstractFlowManager implements FlowMan
     @Override
     public ControllerServiceNode createControllerService(final String type, final String id, final BundleCoordinate bundleCoordinate, final Set<URL> additionalUrls,
                                                          final boolean firstTimeAdded, final boolean registerLogObserver) {
+
         logger.debug("Creating Controller Service of type {} with id {}", type, id);
+        final LogRepository logRepository = LogRepositoryFactory.getRepository(id);
+
         final ControllerServiceNode serviceNode = new ComponentBuilder()
             .identifier(id)
             .type(type)
@@ -337,6 +353,12 @@ public class StatelessFlowManager extends AbstractFlowManager implements FlowMan
             throw new ComponentLifeCycleException("Failed to invoke On-Added Lifecycle methods of " + serviceImpl, e);
         }
 
+        LogRepositoryFactory.getRepository(serviceNode.getIdentifier()).setLogger(serviceNode.getLogger());
+        if (registerLogObserver) {
+            // Register log observer to provide bulletins when reporting task logs anything at WARN level or above
+            logRepository.addObserver(StandardProcessorNode.BULLETIN_OBSERVER_ID, LogLevel.WARN, new ControllerServiceLogObserver(bulletinRepository, serviceNode));
+        }
+
         statelessEngine.getControllerServiceProvider().onControllerServiceAdded(serviceNode);
         logger.debug("Controller Service with id {} successfully created", id);
 
diff --git a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/flow/CanceledTriggerResult.java b/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/flow/CanceledTriggerResult.java
index 69bd256..a17bf99 100644
--- a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/flow/CanceledTriggerResult.java
+++ b/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/flow/CanceledTriggerResult.java
@@ -18,6 +18,7 @@
 package org.apache.nifi.stateless.flow;
 
 import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.provenance.ProvenanceEventRecord;
 
 import java.io.ByteArrayInputStream;
 import java.io.InputStream;
@@ -69,4 +70,9 @@ public class CanceledTriggerResult implements TriggerResult {
     @Override
     public void abort(final Throwable cause) {
     }
+
+    @Override
+    public List<ProvenanceEventRecord> getProvenanceEvents() {
+        return Collections.emptyList();
+    }
 }
diff --git a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/flow/ExceptionalTriggerResult.java b/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/flow/ExceptionalTriggerResult.java
index 35f0107..20db5d1 100644
--- a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/flow/ExceptionalTriggerResult.java
+++ b/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/flow/ExceptionalTriggerResult.java
@@ -19,6 +19,7 @@ package org.apache.nifi.stateless.flow;
 
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.exception.TerminatedTaskException;
+import org.apache.nifi.provenance.ProvenanceEventRecord;
 
 import java.io.IOException;
 import java.io.InputStream;
@@ -79,4 +80,9 @@ public class ExceptionalTriggerResult implements TriggerResult {
             failureCause.addSuppressed(cause);
         }
     }
+
+    @Override
+    public List<ProvenanceEventRecord> getProvenanceEvents() throws IOException {
+        return Collections.emptyList();
+    }
 }
diff --git a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/flow/StandardStatelessDataflowFactory.java b/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/flow/StandardStatelessDataflowFactory.java
index 2440fb6..b15cb58 100644
--- a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/flow/StandardStatelessDataflowFactory.java
+++ b/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/flow/StandardStatelessDataflowFactory.java
@@ -104,8 +104,9 @@ public class StandardStatelessDataflowFactory implements StatelessDataflowFactor
         try {
             final BulletinRepository bulletinRepository = new VolatileBulletinRepository();
             final File workingDir = engineConfiguration.getWorkingDirectory();
-            if (!workingDir.exists() && !workingDir.mkdirs()) {
-                throw new IOException("Working Directory " + workingDir + " does not exist and could not be created");
+            final File narExpansionDirectory = new File(workingDir, "nar");
+            if (!narExpansionDirectory.exists() && !narExpansionDirectory.mkdirs()) {
+                throw new IOException("Working Directory " + narExpansionDirectory + " does not exist and could not be created");
             }
 
             final InMemoryFlowRegistry flowRegistry = new InMemoryFlowRegistry();
@@ -114,9 +115,9 @@ public class StandardStatelessDataflowFactory implements StatelessDataflowFactor
             flowRegistryClient.addFlowRegistry(flowRegistry);
 
             final NarClassLoaders narClassLoaders = new NarClassLoaders();
-            final File extensionsWorkingDir = new File(workingDir, "extensions");
+            final File extensionsWorkingDir = new File(narExpansionDirectory, "extensions");
             final ClassLoader systemClassLoader = createSystemClassLoader(engineConfiguration.getNarDirectory());
-            final ExtensionDiscoveringManager extensionManager = ExtensionDiscovery.discover(extensionsWorkingDir, systemClassLoader, narClassLoaders);
+            final ExtensionDiscoveringManager extensionManager = ExtensionDiscovery.discover(extensionsWorkingDir, systemClassLoader, narClassLoaders, engineConfiguration.isLogExtensionDiscovery());
 
             flowFileEventRepo = new RingBufferEventRepository(5);
 
@@ -174,8 +175,10 @@ public class StandardStatelessDataflowFactory implements StatelessDataflowFactor
 
             final File krb5File = engineConfiguration.getKrb5File();
             final KerberosConfig kerberosConfig = new KerberosConfig(null, null, krb5File);
-            logger.info("Setting java.security.krb5.conf to {}", krb5File.getAbsolutePath());
-            System.setProperty("java.security.krb5.conf", krb5File.getAbsolutePath());
+            if (krb5File != null) {
+                logger.info("Setting java.security.krb5.conf to {}", krb5File.getAbsolutePath());
+                System.setProperty("java.security.krb5.conf", krb5File.getAbsolutePath());
+            }
 
             final StatelessEngine<VersionedFlowSnapshot> statelessEngine = new StandardStatelessEngine.Builder()
                 .bulletinRepository(bulletinRepository)
@@ -192,7 +195,7 @@ public class StandardStatelessDataflowFactory implements StatelessDataflowFactor
                 .counterRepository(counterRepo)
                 .build();
 
-            final StatelessFlowManager flowManager = new StatelessFlowManager(flowFileEventRepo, parameterContextManager, statelessEngine, () -> true, sslContext);
+            final StatelessFlowManager flowManager = new StatelessFlowManager(flowFileEventRepo, parameterContextManager, statelessEngine, () -> true, sslContext, bulletinRepository);
             final ControllerServiceProvider controllerServiceProvider = new StandardControllerServiceProvider(processScheduler, bulletinRepository, flowManager, extensionManager);
 
             final ProcessContextFactory rawProcessContextFactory = new StatelessProcessContextFactory(controllerServiceProvider, lazyInitializedEncryptor, stateManagerProvider);
diff --git a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/flow/StandardStatelessFlow.java b/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/flow/StandardStatelessFlow.java
index b6f9d37..344d747 100644
--- a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/flow/StandardStatelessFlow.java
+++ b/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/flow/StandardStatelessFlow.java
@@ -24,15 +24,18 @@ import org.apache.nifi.components.state.StateMap;
 import org.apache.nifi.components.state.StatelessStateManagerProvider;
 import org.apache.nifi.components.validation.ValidationStatus;
 import org.apache.nifi.connectable.Connectable;
+import org.apache.nifi.connectable.ConnectableType;
 import org.apache.nifi.connectable.Connection;
 import org.apache.nifi.connectable.LocalPort;
 import org.apache.nifi.connectable.Port;
 import org.apache.nifi.controller.ComponentNode;
+import org.apache.nifi.controller.Counter;
 import org.apache.nifi.controller.ProcessScheduler;
 import org.apache.nifi.controller.ProcessorNode;
 import org.apache.nifi.controller.ReportingTaskNode;
 import org.apache.nifi.controller.queue.FlowFileQueue;
 import org.apache.nifi.controller.queue.QueueSize;
+import org.apache.nifi.controller.repository.CounterRepository;
 import org.apache.nifi.controller.repository.FlowFileRecord;
 import org.apache.nifi.controller.repository.RepositoryContext;
 import org.apache.nifi.controller.repository.StandardProcessSessionFactory;
@@ -45,8 +48,10 @@ import org.apache.nifi.groups.RemoteProcessGroup;
 import org.apache.nifi.processor.ProcessSession;
 import org.apache.nifi.processor.ProcessSessionFactory;
 import org.apache.nifi.processor.Processor;
+import org.apache.nifi.processor.exception.FlowFileAccessException;
 import org.apache.nifi.processor.exception.TerminatedTaskException;
 import org.apache.nifi.remote.RemoteGroupPort;
+import org.apache.nifi.reporting.BulletinRepository;
 import org.apache.nifi.stateless.engine.ExecutionProgress;
 import org.apache.nifi.stateless.engine.ExecutionProgress.CompletionAction;
 import org.apache.nifi.stateless.engine.ProcessContextFactory;
@@ -54,10 +59,14 @@ import org.apache.nifi.stateless.engine.StandardExecutionProgress;
 import org.apache.nifi.stateless.queue.DrainableFlowFileQueue;
 import org.apache.nifi.stateless.repository.RepositoryContextFactory;
 import org.apache.nifi.stateless.session.AsynchronousCommitTracker;
+import org.apache.nifi.stream.io.StreamUtils;
 import org.apache.nifi.util.Connectables;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
 import java.text.NumberFormat;
 import java.util.ArrayList;
 import java.util.Collection;
@@ -83,6 +92,7 @@ public class StandardStatelessFlow implements StatelessDataflow {
     private static final Logger logger = LoggerFactory.getLogger(StandardStatelessFlow.class);
     private static final long COMPONENT_ENABLE_TIMEOUT_MILLIS = TimeUnit.SECONDS.toMillis(10);
     private static final long TEN_MILLIS_IN_NANOS = TimeUnit.MILLISECONDS.toNanos(10);
+    private static final String PARENT_FLOW_GROUP_ID = "stateless-flow";
 
     private final ProcessGroup rootGroup;
     private final List<Connection> allConnections;
@@ -99,6 +109,7 @@ public class StandardStatelessFlow implements StatelessDataflow {
     private final AsynchronousCommitTracker tracker = new AsynchronousCommitTracker();
     private final TransactionThresholdMeter transactionThresholdMeter;
     private final List<BackgroundTask> backgroundTasks = new ArrayList<>();
+    private final BulletinRepository bulletinRepository;
 
     private volatile ExecutorService runDataflowExecutor;
     private volatile ScheduledExecutorService backgroundTaskExecutor;
@@ -106,7 +117,7 @@ public class StandardStatelessFlow implements StatelessDataflow {
 
     public StandardStatelessFlow(final ProcessGroup rootGroup, final List<ReportingTaskNode> reportingTasks, final ControllerServiceProvider controllerServiceProvider,
                                  final ProcessContextFactory processContextFactory, final RepositoryContextFactory repositoryContextFactory, final DataflowDefinition<?> dataflowDefinition,
-                                 final StatelessStateManagerProvider stateManagerProvider, final ProcessScheduler processScheduler) {
+                                 final StatelessStateManagerProvider stateManagerProvider, final ProcessScheduler processScheduler, final BulletinRepository bulletinRepository) {
         this.rootGroup = rootGroup;
         this.allConnections = rootGroup.findAllConnections();
         this.reportingTasks = reportingTasks;
@@ -117,6 +128,7 @@ public class StandardStatelessFlow implements StatelessDataflow {
         this.stateManagerProvider = stateManagerProvider;
         this.processScheduler = processScheduler;
         this.transactionThresholdMeter = new TransactionThresholdMeter(dataflowDefinition.getTransactionThresholds());
+        this.bulletinRepository = bulletinRepository;
 
         rootConnectables = new HashSet<>();
 
@@ -145,7 +157,9 @@ public class StandardStatelessFlow implements StatelessDataflow {
         for (final Port port : processGroup.getInputPorts()) {
             for (final Connection connection : port.getConnections()) {
                 final Connectable connectable = connection.getDestination();
-                rootComponents.add(connectable);
+                if (!isTerminalPort(connectable)) {
+                    rootComponents.add(connectable);
+                }
             }
         }
     }
@@ -171,6 +185,21 @@ public class StandardStatelessFlow implements StatelessDataflow {
         }
     }
 
+    public static boolean isTerminalPort(final Connectable connectable) {
+        final ConnectableType connectableType = connectable.getConnectableType();
+        if (connectableType != ConnectableType.OUTPUT_PORT) {
+            return false;
+        }
+
+        final ProcessGroup portGroup = connectable.getProcessGroup();
+        if (PARENT_FLOW_GROUP_ID.equals(portGroup.getIdentifier())) {
+            logger.debug("FlowFiles queued for {} but this is a Terminal Port. Will not trigger Port to run.", connectable);
+            return true;
+        }
+
+        return false;
+    }
+
     @Override
     public void initialize() {
         if (initialized) {
@@ -396,7 +425,7 @@ public class StandardStatelessFlow implements StatelessDataflow {
     }
 
     @Override
-    public DataflowTrigger trigger() {
+    public DataflowTrigger trigger(final DataflowTriggerContext triggerContext) {
         if (!initialized) {
             throw new IllegalStateException("Must initialize dataflow before triggering it");
         }
@@ -404,8 +433,7 @@ public class StandardStatelessFlow implements StatelessDataflow {
         final BlockingQueue<TriggerResult> resultQueue = new LinkedBlockingQueue<>();
 
         final ExecutionProgress executionProgress = new StandardExecutionProgress(rootGroup, internalFlowFileQueues, resultQueue,
-            repositoryContextFactory.getContentRepository(), dataflowDefinition.getFailurePortNames(), tracker,
-            stateManagerProvider);
+            repositoryContextFactory, dataflowDefinition.getFailurePortNames(), tracker, stateManagerProvider, triggerContext);
 
         final AtomicReference<Future<?>> processFuture = new AtomicReference<>();
         final DataflowTrigger trigger = new DataflowTrigger() {
@@ -471,9 +499,11 @@ public class StandardStatelessFlow implements StatelessDataflow {
                     break;
                 case COMPLETE:
                 default:
-                    final long nanos = System.nanoTime() - startNanos;
-                    final String prettyPrinted = (nanos > TEN_MILLIS_IN_NANOS) ? (TimeUnit.NANOSECONDS.toMillis(nanos) + " millis") : NumberFormat.getInstance().format(nanos) + " nanos";
-                    logger.info("Ran dataflow in {}", prettyPrinted);
+                    if (logger.isDebugEnabled()) {
+                        final long nanos = System.nanoTime() - startNanos;
+                        final String prettyPrinted = (nanos > TEN_MILLIS_IN_NANOS) ? (TimeUnit.NANOSECONDS.toMillis(nanos) + " millis") : NumberFormat.getInstance().format(nanos) + " nanos";
+                        logger.debug("Ran dataflow in {}", prettyPrinted);
+                    }
                     break;
             }
         } catch (final TerminatedTaskException tte) {
@@ -509,6 +539,17 @@ public class StandardStatelessFlow implements StatelessDataflow {
 
     @Override
     public QueueSize enqueue(final byte[] flowFileContents, final Map<String, String> attributes, final String portName) {
+        try {
+            try (final InputStream bais = new ByteArrayInputStream(flowFileContents)) {
+                return enqueue(bais, attributes, portName);
+            }
+        } catch (final IOException e) {
+            throw new FlowFileAccessException("Failed to enqueue FlowFile", e);
+        }
+    }
+
+    @Override
+    public QueueSize enqueue(final InputStream flowFileContents, final Map<String, String> attributes, final String portName) {
         final Port inputPort = rootGroup.getInputPortByName(portName);
         if (inputPort == null) {
             throw new IllegalArgumentException("No Input Port exists with name <" + portName + ">. Valid Port names are " + getInputPortNames());
@@ -526,7 +567,7 @@ public class StandardStatelessFlow implements StatelessDataflow {
             }
 
             FlowFile flowFile = session.create();
-            flowFile = session.write(flowFile, out -> out.write(flowFileContents));
+            flowFile = session.write(flowFile, out -> StreamUtils.copy(flowFileContents, out));
             flowFile = session.putAllAttributes(flowFile, attributes);
             session.transfer(flowFile, LocalPort.PORT_RELATIONSHIP);
             session.commitAsync();
@@ -652,6 +693,33 @@ public class StandardStatelessFlow implements StatelessDataflow {
         return latest;
     }
 
+    @Override
+    public void resetCounters() {
+        final CounterRepository counterRepo = repositoryContextFactory.getCounterRepository();
+        counterRepo.getCounters().forEach(counter -> counterRepo.resetCounter(counter.getIdentifier()));
+    }
+
+    @Override
+    public Map<String, Long> getCounters(final boolean includeGlobalContext) {
+        final Map<String, Long> counters = new HashMap<>();
+        for (final Counter counter : repositoryContextFactory.getCounterRepository().getCounters()) {
+            // Counter context is either of the format `componentName (componentId)` or `All componentType's` (global context). We only want the
+            // those of the first type - for individual components - unless includeGlobalContext == true
+            final boolean isGlobalContext = !counter.getContext().endsWith(")");
+            if (includeGlobalContext || !isGlobalContext) {
+                final String counterName = isGlobalContext ? counter.getName() : (counter.getName() + " - " + counter.getContext());
+                counters.put(counterName, counter.getValue());
+            }
+        }
+
+        return counters;
+    }
+
+    @Override
+    public BulletinRepository getBulletinRepository() {
+        return bulletinRepository;
+    }
+
     @SuppressWarnings("unused")
     public Set<Processor> findAllProcessors() {
         return rootGroup.findAllProcessors().stream()
diff --git a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/flow/StandardStatelessFlowCurrent.java b/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/flow/StandardStatelessFlowCurrent.java
index 11496c7..099f9f8 100644
--- a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/flow/StandardStatelessFlowCurrent.java
+++ b/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/flow/StandardStatelessFlowCurrent.java
@@ -18,8 +18,11 @@
 package org.apache.nifi.stateless.flow;
 
 import org.apache.nifi.connectable.Connectable;
+import org.apache.nifi.connectable.ConnectableType;
 import org.apache.nifi.controller.repository.metrics.StandardFlowFileEvent;
+import org.apache.nifi.groups.FlowFileOutboundPolicy;
 import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.exception.TerminatedTaskException;
 import org.apache.nifi.stateless.engine.ExecutionProgress;
 import org.apache.nifi.stateless.engine.ProcessContextFactory;
 import org.apache.nifi.stateless.repository.RepositoryContextFactory;
@@ -91,7 +94,12 @@ public class StandardStatelessFlowCurrent implements StatelessFlowCurrent {
                 completionReached = !tracker.isAnyReady();
             }
         } catch (final Throwable t) {
-            logger.error("Failed to trigger {}", currentComponent, t);
+            if (t instanceof TerminatedTaskException) {
+                logger.debug("Encountered TerminatedTaskException when triggering {}", currentComponent, t);
+            } else {
+                logger.error("Failed to trigger {}", currentComponent, t);
+            }
+
             executionProgress.notifyExecutionFailed(t);
             tracker.triggerFailureCallbacks(t);
             throw t;
@@ -135,6 +143,17 @@ public class StandardStatelessFlowCurrent implements StatelessFlowCurrent {
                 continue;
             }
 
+            // If we've made no progress, check the condition of this being an Output Port with Batch Output. In such a case, we will make no progress
+            // until data has been processed elsewhere in the flow, so return NEXT_READY.
+            if (connectable.getConnectableType() == ConnectableType.OUTPUT_PORT && connectable.getProcessGroup().getFlowFileOutboundPolicy() == FlowFileOutboundPolicy.BATCH_OUTPUT
+                    && connectable.getProcessGroup().isDataQueuedForProcessing()) {
+
+                logger.debug("{} was triggered but unable to make process. Data is still available for processing, so continue triggering components within the Process Group", connectable);
+                return NextConnectable.NEXT_READY;
+            }
+
+            // Check if we've reached out threshold for how much data we are willing to bring into a single transaction. If so, we will not drop back to
+            // triggering source components
             final boolean thresholdMet = transactionThresholdMeter.isThresholdMet();
             if (thresholdMet) {
                 logger.debug("{} was triggered but unable to make progress. The transaction thresholds {} have been met (currently at {}). Will not " +
diff --git a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/repository/RepositoryContextFactory.java b/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/repository/RepositoryContextFactory.java
index 827e75c..b243eb6 100644
--- a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/repository/RepositoryContextFactory.java
+++ b/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/repository/RepositoryContextFactory.java
@@ -19,8 +19,10 @@ package org.apache.nifi.stateless.repository;
 
 import org.apache.nifi.connectable.Connectable;
 import org.apache.nifi.controller.repository.ContentRepository;
+import org.apache.nifi.controller.repository.CounterRepository;
 import org.apache.nifi.controller.repository.FlowFileEventRepository;
 import org.apache.nifi.controller.repository.RepositoryContext;
+import org.apache.nifi.provenance.ProvenanceEventRepository;
 
 public interface RepositoryContextFactory {
     RepositoryContext createRepositoryContext(Connectable connectable);
@@ -29,5 +31,9 @@ public interface RepositoryContextFactory {
 
     FlowFileEventRepository getFlowFileEventRepository();
 
+    ProvenanceEventRepository getProvenanceRepository();
+
+    CounterRepository getCounterRepository();
+
     void shutdown();
 }
diff --git a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/repository/StatelessFileSystemContentRepository.java b/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/repository/StatelessFileSystemContentRepository.java
index 8a4e885..e1c8e0d 100644
--- a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/repository/StatelessFileSystemContentRepository.java
+++ b/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/repository/StatelessFileSystemContentRepository.java
@@ -350,6 +350,7 @@ public class StatelessFileSystemContentRepository implements ContentRepository {
         private final StandardContentClaim scc;
         private final SynchronizedByteCountingOutputStream out;
         private final long initialOffset;
+        private boolean closed = false;
 
         public ContentOutputStream(final SynchronizedByteCountingOutputStream out, final StandardContentClaim scc) {
             super(out);
@@ -360,6 +361,11 @@ public class StatelessFileSystemContentRepository implements ContentRepository {
 
         @Override
         public synchronized void close() throws IOException {
+            if (closed) {
+                return;
+            }
+
+            closed = true;
             super.flush();
             scc.setLength(out.getBytesWritten() - initialOffset);
             writableClaimQueue.offer(scc.getResourceClaim());
diff --git a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/repository/StatelessRepositoryContextFactory.java b/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/repository/StatelessRepositoryContextFactory.java
index 21d35f5..6152081 100644
--- a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/repository/StatelessRepositoryContextFactory.java
+++ b/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/repository/StatelessRepositoryContextFactory.java
@@ -70,6 +70,16 @@ public class StatelessRepositoryContextFactory implements RepositoryContextFacto
     }
 
     @Override
+    public ProvenanceEventRepository getProvenanceRepository() {
+        return provenanceEventRepository;
+    }
+
+    @Override
+    public CounterRepository getCounterRepository() {
+        return counterRepository;
+    }
+
+    @Override
     public void shutdown() {
         contentRepository.shutdown();
 
diff --git a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/session/StatelessProcessSession.java b/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/session/StatelessProcessSession.java
index 2b36b8f..1002ab7 100644
--- a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/session/StatelessProcessSession.java
+++ b/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/session/StatelessProcessSession.java
@@ -29,6 +29,8 @@ import org.apache.nifi.processor.ProcessSessionFactory;
 import org.apache.nifi.stateless.engine.DataflowAbortedException;
 import org.apache.nifi.stateless.engine.ExecutionProgress;
 import org.apache.nifi.stateless.engine.ProcessContextFactory;
+import org.apache.nifi.stateless.flow.FailurePortEncounteredException;
+import org.apache.nifi.stateless.flow.StandardStatelessFlow;
 import org.apache.nifi.stateless.queue.DrainableFlowFileQueue;
 import org.apache.nifi.stateless.repository.RepositoryContextFactory;
 import org.slf4j.Logger;
@@ -41,7 +43,6 @@ import java.util.function.Consumer;
 
 public class StatelessProcessSession extends StandardProcessSession {
     private static final Logger logger = LoggerFactory.getLogger(StatelessProcessSession.class);
-    private static final String PARENT_FLOW_GROUP_ID = "stateless-flow";
 
     private final Connectable connectable;
     private final RepositoryContextFactory repositoryContextFactory;
@@ -163,7 +164,11 @@ public class StatelessProcessSession extends StandardProcessSession {
             // until they have consumed all created FlowFiles.
             while (!connection.getFlowFileQueue().isEmpty()) {
                 final Connectable connectable = connection.getDestination();
-                if (isTerminalPort(connectable)) {
+                if (isFailurePortGuaranteed(connectable)) {
+                    throw new FailurePortEncounteredException("FlowFile was transferred to Port " + connectable.getName() + ", which is marked as a Failure Port", connectable.getName());
+                }
+
+                if (StandardStatelessFlow.isTerminalPort(connectable)) {
                     // If data is being transferred to a terminal port, we don't want to trigger the port,
                     // as it has nowhere to transfer the data. We simply leave it queued at the terminal port.
                     // Once the processing completes, the terminal ports' connections will be drained, when #awaitAcknowledgment is called.
@@ -185,7 +190,11 @@ public class StatelessProcessSession extends StandardProcessSession {
             }
 
             final Connectable connectable = connection.getDestination();
-            if (isTerminalPort(connectable)) {
+            if (isFailurePortGuaranteed(connectable)) {
+                throw new FailurePortEncounteredException("FlowFile was transferred to Port " + connectable.getName() + ", which is marked as a Failure Port", connectable.getName());
+            }
+
+            if (StandardStatelessFlow.isTerminalPort(connectable)) {
                 // If data is being transferred to a terminal port, we don't want to trigger the port,
                 // as it has nowhere to transfer the data. We simply leave it queued at the terminal port.
                 // Once the processing completes, the terminal ports' connections will be drained, when #awaitAcknowledgment is called.
@@ -196,6 +205,33 @@ public class StatelessProcessSession extends StandardProcessSession {
         }
     }
 
+    /**
+     * Determines whether or not the given Port is a failure port or if transferring to the connectable guarantees failure.
+     * This allows us to have an inner Process Group with an Output Port, for instance, named 'failure' that is connected to a 'failure' port at
+     * a higher level. In this case, transferring to the inner group's 'failure' port guarantees failure, so this method returns true.
+     *
+     * @param connectable the connectable that may or may not be a failure port
+     * @return <code>true</code> if transferring to the given Connectable guarantees dataflow failure, <code>false</code> otherwise.
+     */
+    private boolean isFailurePortGuaranteed(final Connectable connectable) {
+        final ConnectableType connectableType = connectable.getConnectableType();
+        if (connectableType != ConnectableType.OUTPUT_PORT && connectableType != ConnectableType.FUNNEL) {
+            return false;
+        }
+
+        if (executionProgress.isFailurePort(connectable.getName())) {
+            return true;
+        }
+
+        for (final Connection outboundConnection : connectable.getConnections()) {
+            if (isFailurePortGuaranteed(outboundConnection.getDestination())) {
+                return true;
+            }
+        }
+
+        return false;
+    }
+
     private void triggerNext(final Connectable connectable) {
         assertProgressNotCanceled();
 
@@ -300,21 +336,6 @@ public class StatelessProcessSession extends StandardProcessSession {
         }
     }
 
-    private boolean isTerminalPort(final Connectable connectable) {
-        final ConnectableType connectableType = connectable.getConnectableType();
-        if (connectableType != ConnectableType.OUTPUT_PORT) {
-            return false;
-        }
-
-        final ProcessGroup portGroup = connectable.getProcessGroup();
-        if (PARENT_FLOW_GROUP_ID.equals(portGroup.getIdentifier())) {
-            logger.debug("FlowFiles queued for {} but this is a Terminal Port. Will not trigger Port to run.", connectable);
-            return true;
-        }
-
-        return false;
-    }
-
     @Override
     public String toString() {
         return "StatelessProcessSession[id=" + getSessionId() + "]";
diff --git a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/test/java/org/apache/nifi/stateless/repository/TestStatelessFileSystemContentRepository.java b/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/test/java/org/apache/nifi/stateless/repository/TestStatelessFileSystemContentRepository.java
index d7b5d54..c6bba2f 100644
--- a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/test/java/org/apache/nifi/stateless/repository/TestStatelessFileSystemContentRepository.java
+++ b/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/test/java/org/apache/nifi/stateless/repository/TestStatelessFileSystemContentRepository.java
@@ -18,6 +18,7 @@
 package org.apache.nifi.stateless.repository;
 
 import org.apache.nifi.controller.repository.claim.ContentClaim;
+import org.apache.nifi.controller.repository.claim.ResourceClaim;
 import org.apache.nifi.controller.repository.claim.StandardResourceClaimManager;
 import org.apache.nifi.stream.io.StreamUtils;
 import org.junit.jupiter.api.AfterEach;
@@ -29,7 +30,9 @@ import java.io.File;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
+import java.util.Arrays;
 
+import static org.junit.Assert.assertTrue;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertNotNull;
 import static org.junit.jupiter.api.Assertions.assertThrows;
@@ -105,4 +108,50 @@ public class TestStatelessFileSystemContentRepository {
         }
     }
 
+    @Test
+    public void testWriteToMultipleStreams() throws IOException {
+        final ContentClaim claim1 = repository.create(true);
+        final ContentClaim claim2 = repository.create(true);
+        final ContentClaim claim3 = repository.create(true);
+
+        final OutputStream out1 = repository.write(claim1);
+        final OutputStream out2 = repository.write(claim2);
+        final OutputStream out3 = repository.write(claim3);
+
+        for (final char c : "Hello World".toCharArray()) {
+            out1.write(c);
+            out2.write(c);
+            out3.write(c);
+        }
+
+        out1.close();
+        out2.close();
+        out3.close();
+
+        for (final ContentClaim claim : Arrays.asList(claim1, claim2, claim3)) {
+            try (final InputStream in = repository.read(claim)) {
+                for (final char c : "Hello World".toCharArray()) {
+                    assertEquals(c, in.read());
+                }
+
+                assertEquals(-1, in.read());
+            }
+        }
+
+        final ContentClaim claim4 = repository.create(true);
+        final ResourceClaim resourceClaim4 = claim4.getResourceClaim();
+        assertTrue(resourceClaim4.equals(claim1.getResourceClaim()) || resourceClaim4.equals(claim2.getResourceClaim()) || resourceClaim4.equals(claim3.getResourceClaim()));
+
+        try (final OutputStream out4 = repository.write(claim4)) {
+            out4.write("Hello World".getBytes());
+        }
+
+        try (final InputStream in = repository.read(claim4)) {
+            for (final char c : "Hello World".toCharArray()) {
+                assertEquals(c, in.read());
+            }
+
+            assertEquals(-1, in.read());
+        }
+    }
 }
diff --git a/nifi-system-tests/nifi-stateless-system-test-suite/src/test/java/org/apache/nifi/stateless/StatelessSystemIT.java b/nifi-system-tests/nifi-stateless-system-test-suite/src/test/java/org/apache/nifi/stateless/StatelessSystemIT.java
index 9691ddd..603d051 100644
--- a/nifi-system-tests/nifi-stateless-system-test-suite/src/test/java/org/apache/nifi/stateless/StatelessSystemIT.java
+++ b/nifi-system-tests/nifi-stateless-system-test-suite/src/test/java/org/apache/nifi/stateless/StatelessSystemIT.java
@@ -96,10 +96,7 @@ public class StatelessSystemIT {
 
             @Override
             public Optional<File> getContentRepositoryDirectory() {
-                return Optional.empty();
-
-                // Can be used to enable file-based content repository.
-//                return Optional.of(new File("target/nifi-stateless-content-repo"));
+                return getContentRepoDirectory();
             }
 
             @Override
@@ -119,6 +116,10 @@ public class StatelessSystemIT {
         };
     }
 
+    protected Optional<File> getContentRepoDirectory() {
+        return Optional.empty();
+    }
+
     protected StatelessDataflow loadDataflow(final File versionedFlowSnapshot, final List<ParameterContextDefinition> parameterContexts) throws IOException, StatelessConfigurationException {
         final ObjectMapper objectMapper = new ObjectMapper();
 
diff --git a/nifi-system-tests/nifi-stateless-system-test-suite/src/test/java/org/apache/nifi/stateless/basics/BatchOutputIT.java b/nifi-system-tests/nifi-stateless-system-test-suite/src/test/java/org/apache/nifi/stateless/basics/BatchOutputIT.java
new file mode 100644
index 0000000..1a06c31
--- /dev/null
+++ b/nifi-system-tests/nifi-stateless-system-test-suite/src/test/java/org/apache/nifi/stateless/basics/BatchOutputIT.java
@@ -0,0 +1,120 @@
+/*
+ * 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.stateless.basics;
+
+import org.apache.nifi.flow.VersionedPort;
+import org.apache.nifi.flow.VersionedProcessGroup;
+import org.apache.nifi.flow.VersionedProcessor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.stateless.StatelessSystemIT;
+import org.apache.nifi.stateless.VersionedFlowBuilder;
+import org.apache.nifi.stateless.config.StatelessConfigurationException;
+import org.apache.nifi.stateless.flow.DataflowTrigger;
+import org.apache.nifi.stateless.flow.StatelessDataflow;
+import org.apache.nifi.stateless.flow.TriggerResult;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.Collections;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class BatchOutputIT extends StatelessSystemIT {
+
+    @Test
+    public void testOutputOfSingleFlowFileRuns() throws IOException, StatelessConfigurationException, InterruptedException {
+        // Build the flow
+        final VersionedFlowBuilder flowBuilder = new VersionedFlowBuilder();
+        final VersionedProcessGroup rootGroup = flowBuilder.getRootGroup();
+
+        final VersionedProcessGroup innerGroup = flowBuilder.createProcessGroup("Inner", rootGroup);
+        innerGroup.setFlowFileOutboundPolicy("BATCH_OUTPUT");
+
+        final VersionedProcessor generate = flowBuilder.createSimpleProcessor("GenerateFlowFile", innerGroup);
+        generate.setProperties(Collections.singletonMap("Text", "Hello World"));
+        final VersionedPort outputPort = flowBuilder.createOutputPort("Out", innerGroup);
+        flowBuilder.createConnection(generate, outputPort, "success", innerGroup);
+
+        final VersionedPort fin = flowBuilder.createOutputPort("Fin");
+        flowBuilder.createConnection(outputPort, fin, "", rootGroup);
+
+        // Startup the dataflow allowing a Transaction Threshold large enough to accommodate
+        final StatelessDataflow dataflow = loadDataflow(flowBuilder.getFlowSnapshot());
+
+        // Enqueue data and trigger
+        final DataflowTrigger trigger = dataflow.trigger();
+        final TriggerResult result = trigger.getResult();
+        assertTrue(result.isSuccessful());
+
+        final List<FlowFile> outputFlowFiles = result.getOutputFlowFiles("Fin");
+        assertEquals(1, outputFlowFiles.size());
+
+        final String outputText = new String(result.readContentAsByteArray(outputFlowFiles.get(0)), StandardCharsets.UTF_8);
+        assertEquals("Hello World", outputText);
+    }
+
+    @Test
+    public void testWaitsForAvailableFlowFiles() throws IOException, StatelessConfigurationException, InterruptedException {
+        // Build the flow
+        final VersionedFlowBuilder flowBuilder = new VersionedFlowBuilder();
+        final VersionedProcessGroup rootGroup = flowBuilder.getRootGroup();
+
+        final VersionedProcessGroup innerGroup = flowBuilder.createProcessGroup("Inner", rootGroup);
+        innerGroup.setFlowFileOutboundPolicy("BATCH_OUTPUT");
+
+        final VersionedProcessor generate = flowBuilder.createSimpleProcessor("GenerateFlowFile", innerGroup);
+        generate.setProperties(Collections.singletonMap("Batch Size", "100"));
+
+        final VersionedProcessor route = flowBuilder.createSimpleProcessor("RoundRobinFlowFiles", innerGroup);
+        route.setProperties(Collections.singletonMap("Number of Relationships", "2"));
+
+        final VersionedProcessor sleep1 = flowBuilder.createSimpleProcessor("Sleep", innerGroup);
+        sleep1.setProperties(Collections.singletonMap("onTrigger Sleep Time", "5 ms"));
+
+        final VersionedProcessor sleep2 = flowBuilder.createSimpleProcessor("Sleep", innerGroup);
+        sleep2.setProperties(Collections.singletonMap("onTrigger Sleep Time", "5 ms"));
+
+        final VersionedProcessor sleep3 = flowBuilder.createSimpleProcessor("Sleep", innerGroup);
+        sleep3.setProperties(Collections.singletonMap("onTrigger Sleep Time", "5 ms"));
+
+        final VersionedPort outputPort = flowBuilder.createOutputPort("Out", innerGroup);
+        flowBuilder.createConnection(generate, route, "success", innerGroup);
+        flowBuilder.createConnection(route, outputPort, "1", innerGroup);
+        flowBuilder.createConnection(route, sleep1, "2", innerGroup);
+        flowBuilder.createConnection(sleep1, sleep2, "success", innerGroup);
+        flowBuilder.createConnection(sleep2, sleep3, "success", innerGroup);
+        flowBuilder.createConnection(sleep3, outputPort, "success", innerGroup);
+
+        final VersionedPort fin = flowBuilder.createOutputPort("Fin");
+        flowBuilder.createConnection(outputPort, fin, "", rootGroup);
+
+        // Startup the dataflow allowing a Transaction Threshold large enough to accommodate
+        final StatelessDataflow dataflow = loadDataflow(flowBuilder.getFlowSnapshot());
+
+        // Enqueue data and trigger
+        final DataflowTrigger trigger = dataflow.trigger();
+        final TriggerResult result = trigger.getResult();
+        assertTrue(result.isSuccessful());
+
+        final List<FlowFile> outputFlowFiles = result.getOutputFlowFiles("Fin");
+        assertEquals(100, outputFlowFiles.size());
+    }
+}
diff --git a/nifi-system-tests/nifi-stateless-system-test-suite/src/test/java/org/apache/nifi/stateless/basics/PartitioningContentIT.java b/nifi-system-tests/nifi-stateless-system-test-suite/src/test/java/org/apache/nifi/stateless/basics/PartitioningContentIT.java
new file mode 100644
index 0000000..fe7073d
--- /dev/null
+++ b/nifi-system-tests/nifi-stateless-system-test-suite/src/test/java/org/apache/nifi/stateless/basics/PartitioningContentIT.java
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.stateless.basics;
+
+import org.apache.nifi.flow.VersionedPort;
+import org.apache.nifi.flow.VersionedProcessor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.stateless.StatelessSystemIT;
+import org.apache.nifi.stateless.VersionedFlowBuilder;
+import org.apache.nifi.stateless.config.StatelessConfigurationException;
+import org.apache.nifi.stateless.flow.DataflowTrigger;
+import org.apache.nifi.stateless.flow.StatelessDataflow;
+import org.apache.nifi.stateless.flow.TriggerResult;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class PartitioningContentIT extends StatelessSystemIT {
+
+    @Override
+    protected Optional<File> getContentRepoDirectory() {
+        return Optional.of(new File("target/content-repo"));
+    }
+
+    @Test
+    public void testPartitionThenUpdate() throws IOException, StatelessConfigurationException, InterruptedException {
+        final VersionedFlowBuilder flowBuilder = new VersionedFlowBuilder();
+        final VersionedPort outPort = flowBuilder.createOutputPort("Out");
+
+        final VersionedProcessor generate = flowBuilder.createSimpleProcessor("GenerateFlowFile");
+        final Map<String, String> generateProperties = new HashMap<>();
+        generateProperties.put("Text", "abc\n123\nxyz\n321");
+        generateProperties.put("Batch Size", "1");
+        generate.setProperties(generateProperties);
+
+        final VersionedProcessor partition = flowBuilder.createSimpleProcessor("PartitionText");
+
+        flowBuilder.createConnection(generate, partition, "success");
+        flowBuilder.createConnection(partition, outPort, "success");
+
+        // Startup the dataflow
+        final StatelessDataflow dataflow = loadDataflow(flowBuilder.getFlowSnapshot(), Collections.emptyList());
+
+        // Enqueue data and trigger
+        final DataflowTrigger trigger = dataflow.trigger();
+        final TriggerResult result = trigger.getResult();
+        assertTrue(result.isSuccessful());
+
+        final List<FlowFile> flowFiles = result.getOutputFlowFiles("Out");
+        assertEquals(2, flowFiles.size());
+
+        final String[] partitionedContent = new String[] {"abc\nxyz\n", "123\n321\n"};
+        final String[] expectedContent = partitionedContent;
+        for (int i=0; i < expectedContent.length; i++) {
+            final String expected = expectedContent[i];
+
+            final FlowFile flowFile = flowFiles.get(i);
+            final String outputContent = new String(result.readContentAsByteArray(flowFile));
+            assertEquals(expected, outputContent);
+        }
+
+        result.acknowledge();
+    }
+}
diff --git a/nifi-system-tests/nifi-stateless-system-test-suite/src/test/java/org/apache/nifi/stateless/basics/RollbackOnExceptionIT.java b/nifi-system-tests/nifi-stateless-system-test-suite/src/test/java/org/apache/nifi/stateless/basics/RollbackOnExceptionIT.java
index 29ba111..8d2eb62 100644
--- a/nifi-system-tests/nifi-stateless-system-test-suite/src/test/java/org/apache/nifi/stateless/basics/RollbackOnExceptionIT.java
+++ b/nifi-system-tests/nifi-stateless-system-test-suite/src/test/java/org/apache/nifi/stateless/basics/RollbackOnExceptionIT.java
@@ -38,7 +38,7 @@ import static org.junit.Assert.assertTrue;
 public class RollbackOnExceptionIT extends StatelessSystemIT {
     private static final String EXCEPTION_TEXT = "Intentional Exception to verify behavior in RollbackOnExceptionIT";
 
-    @Test
+    @Test(timeout = 30_000)
     public void testFlowFileCompletelyRemovedWhenExceptionThrown() throws IOException, StatelessConfigurationException, InterruptedException {
         final VersionedFlowBuilder builder = new VersionedFlowBuilder();
         final VersionedProcessor generate = builder.createSimpleProcessor("GenerateFlowFile");
@@ -56,6 +56,10 @@ public class RollbackOnExceptionIT extends StatelessSystemIT {
         assertFalse(result.isSuccessful());
         assertTrue(result.getFailureCause().get() instanceof ProcessException);
 
+        // Wait for dataflow to be purged
+        while (dataflow.isFlowFileQueued()) {
+            Thread.sleep(10L);
+        }
         assertFalse(dataflow.isFlowFileQueued());
     }
 
diff --git a/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/processors/tests/system/PartitionText.java b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/processors/tests/system/PartitionText.java
new file mode 100644
index 0000000..63c07c1
--- /dev/null
+++ b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/processors/tests/system/PartitionText.java
@@ -0,0 +1,119 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.processors.tests.system;
+
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.PropertyDescriptor.Builder;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+
+import java.io.BufferedReader;
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.io.Reader;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static org.apache.nifi.expression.ExpressionLanguageScope.FLOWFILE_ATTRIBUTES;
+import static org.apache.nifi.processor.util.StandardValidators.POSITIVE_INTEGER_VALIDATOR;
+
+public class PartitionText extends AbstractProcessor {
+    static final PropertyDescriptor OUTPUT_FLOWFILES = new Builder()
+        .name("Number of Output FlowFiles")
+        .displayName("Number of Output FlowFiles")
+        .description("The number of Output FlowFiles")
+        .required(true)
+        .addValidator(POSITIVE_INTEGER_VALIDATOR)
+        .expressionLanguageSupported(FLOWFILE_ATTRIBUTES)
+        .defaultValue("2")
+        .build();
+
+    static final Relationship REL_SUCCESS = new Relationship.Builder()
+        .name("success")
+        .build();
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return Collections.singletonList(OUTPUT_FLOWFILES);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return Collections.singleton(REL_SUCCESS);
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final int outputCount = context.getProperty(OUTPUT_FLOWFILES).evaluateAttributeExpressions(flowFile).asInteger();
+        final Map<Integer, FlowFile> outputFlowFiles = new HashMap<>();
+        final Map<Integer, OutputStream> outputStreams = new HashMap<>();
+
+        try {
+            try (final InputStream in = session.read(flowFile);
+                 final Reader streamReader = new InputStreamReader(in);
+                 final BufferedReader reader = new BufferedReader(streamReader)) {
+
+                long lineCount = 0L;
+
+                String line;
+                while ((line = reader.readLine()) != null) {
+                    final int flowFileIndex = (int) (lineCount++ % outputCount);
+
+                    OutputStream out = outputStreams.get(flowFileIndex);
+                    if (out == null) {
+                        FlowFile outputFlowFile = session.create(flowFile);
+                        out = session.write(outputFlowFile);
+                        outputFlowFiles.put(flowFileIndex, outputFlowFile);
+                        outputStreams.put(flowFileIndex, out);
+                    }
+
+                    out.write(line.getBytes());
+                    out.write("\n".getBytes());
+                }
+            }
+
+            outputStreams.values().forEach(this::closeQuietly);
+            session.transfer(outputFlowFiles.values(), REL_SUCCESS);
+            session.remove(flowFile);
+        } catch (IOException e) {
+            throw new ProcessException(e);
+        }
+    }
+
+    private void closeQuietly(final Closeable closeable) {
+        try {
+            closeable.close();
+        } catch (final Exception ignored) {
+        }
+    }
+}
diff --git a/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/processors/tests/system/RoundRobinFlowFiles.java b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/processors/tests/system/RoundRobinFlowFiles.java
new file mode 100644
index 0000000..2f07135
--- /dev/null
+++ b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/processors/tests/system/RoundRobinFlowFiles.java
@@ -0,0 +1,87 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.processors.tests.system;
+
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.PropertyDescriptor.Builder;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.nifi.processor.util.StandardValidators.POSITIVE_INTEGER_VALIDATOR;
+
+public class RoundRobinFlowFiles extends AbstractProcessor {
+    private volatile List<Relationship> relationships = new ArrayList<>();
+    private final AtomicLong counter = new AtomicLong(0L);
+
+    static final PropertyDescriptor RELATIONSHIP_COUNT = new Builder()
+        .name("Number of Relationships")
+        .displayName("Number of Relationships")
+        .description("The number of Relationships")
+        .required(true)
+        .addValidator(POSITIVE_INTEGER_VALIDATOR)
+        .build();
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return Collections.singletonList(RELATIONSHIP_COUNT);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return new HashSet<>(relationships);
+    }
+
+    @Override
+    public void onPropertyModified(final PropertyDescriptor descriptor, final String oldValue, final String newValue) {
+        final List<Relationship> relationships = new ArrayList<>();
+        for (int i = 1; i <= Integer.parseInt(newValue); i++) {
+            relationships.add(createRelationship(i));
+        }
+        this.relationships = Collections.unmodifiableList(relationships);
+    }
+
+    private static Relationship createRelationship(final int num) {
+        return new Relationship.Builder().name(String.valueOf(num))
+            .description("Where to route flowfiles for this relationship index").build();
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final long count = counter.getAndIncrement();
+        final long numRelationships = context.getProperty(RELATIONSHIP_COUNT).asLong();
+        final int relationshipIdx = (int) (count % numRelationships);
+        final Relationship relationship = relationships.get(relationshipIdx);
+        session.transfer(flowFile, relationship);
+    }
+}
diff --git a/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/processors/tests/system/TransferBatch.java b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/processors/tests/system/TransferBatch.java
new file mode 100644
index 0000000..5a7520e
--- /dev/null
+++ b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/processors/tests/system/TransferBatch.java
@@ -0,0 +1,102 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.processors.tests.system;
+
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.PropertyDescriptor.Builder;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.nifi.expression.ExpressionLanguageScope.NONE;
+import static org.apache.nifi.processor.util.StandardValidators.POSITIVE_INTEGER_VALIDATOR;
+
+public class TransferBatch extends AbstractProcessor {
+
+    static final AllowableValue ROUTE_TO_FAILURE = new AllowableValue("Route to Failure", "Route to Failure",
+        "If there are not enough FlowFiles available to satisfy the batch size, whatever is available will be transferred to the 'failure' relationship");
+    static final AllowableValue ROLLBACK = new AllowableValue("Rollback", "Rollback",
+        "If there are not enough FlowFiles available to satisfy the batch size, no FlowFiles will be transferred");
+    static final AllowableValue TRANSFER_AVAILABLE = new AllowableValue("Transfer Available", "Transfer Available",
+        "If there are not enough FlowFiles available to satisfy the batch size, whatever is available will be transferred to the 'success' relationship");
+
+
+    static final PropertyDescriptor BATCH_SIZE = new Builder()
+        .name("Batch Size")
+        .displayName("Batch Size")
+        .description("The number of FlowFiles to transfer at once.")
+        .required(true)
+        .addValidator(POSITIVE_INTEGER_VALIDATOR)
+        .expressionLanguageSupported(NONE)
+        .defaultValue("1")
+        .build();
+    static final PropertyDescriptor INSUFFICIENT_BATCH_SIZE_STRATEGY = new Builder()
+        .name("Insufficient Batch Size Strategy")
+        .displayName("Insufficient Batch Size Strategy")
+        .description("Specifies how to handle the situation in which there are fewer FlowFiles available than the configured Batch Size")
+        .required(true)
+        .allowableValues(TRANSFER_AVAILABLE, ROUTE_TO_FAILURE, ROLLBACK)
+        .defaultValue(TRANSFER_AVAILABLE.getValue())
+        .build();
+
+    static final Relationship REL_SUCCESS = new Relationship.Builder()
+        .name("success")
+        .build();
+
+    static final Relationship REL_FAILURE = new Relationship.Builder()
+        .name("failure")
+        .build();
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return Arrays.asList(BATCH_SIZE, INSUFFICIENT_BATCH_SIZE_STRATEGY);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return new HashSet<>(Arrays.asList(REL_SUCCESS, REL_FAILURE));
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final int batchSize = context.getProperty(BATCH_SIZE).asInteger();
+        final List<FlowFile> flowFiles = session.get(batchSize);
+
+        if (flowFiles.size() < batchSize) {
+            final String batchSizeStrategy = context.getProperty(INSUFFICIENT_BATCH_SIZE_STRATEGY).getValue();
+            if (batchSizeStrategy.equalsIgnoreCase(ROUTE_TO_FAILURE.getValue())) {
+                session.transfer(flowFiles, REL_FAILURE);
+            } else if (batchSizeStrategy.equalsIgnoreCase(TRANSFER_AVAILABLE.getValue())) {
+                session.transfer(flowFiles, REL_SUCCESS);
+            } else {
+                session.rollback(false);
+            }
+        } else {
+            session.transfer(flowFiles, REL_SUCCESS);
+        }
+    }
+}
diff --git a/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
index cb653cd..71bcb3e 100644
--- a/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
+++ b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
@@ -26,14 +26,17 @@ org.apache.nifi.processors.tests.system.FakeDynamicPropertiesProcessor
 org.apache.nifi.processors.tests.system.GenerateFlowFile
 org.apache.nifi.processors.tests.system.IngestFile
 org.apache.nifi.processors.tests.system.LoopFlowFile
+org.apache.nifi.processors.tests.system.PartitionText
 org.apache.nifi.processors.tests.system.PassThrough
 org.apache.nifi.processors.tests.system.PassThroughRequiresInstanceClassLoading
 org.apache.nifi.processors.tests.system.ReplaceWithFile
 org.apache.nifi.processors.tests.system.ReverseContents
+org.apache.nifi.processors.tests.system.RoundRobinFlowFiles
 org.apache.nifi.processors.tests.system.SetAttribute
 org.apache.nifi.processors.tests.system.Sleep
 org.apache.nifi.processors.tests.system.SplitByLine
 org.apache.nifi.processors.tests.system.TerminateFlowFile
+org.apache.nifi.processors.tests.system.TransferBatch
 org.apache.nifi.processors.tests.system.ThrowProcessException
 org.apache.nifi.processors.tests.system.ValidateFileExists
 org.apache.nifi.processors.tests.system.VerifyContents