You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by GitBox <gi...@apache.org> on 2021/09/23 17:04:24 UTC

[GitHub] [nifi] markap14 opened a new pull request #5412: NIFI-9239: Updated Consume/Publish Kafka processors to support Exactl…

markap14 opened a new pull request #5412:
URL: https://github.com/apache/nifi/pull/5412


   …y 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.
   
   <!--
     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.
   -->
   Thank you for submitting a contribution to Apache NiFi.
   
   Please provide a short description of the PR here:
   
   #### Description of PR
   
   _Enables X functionality; fixes bug NIFI-YYYY._
   
   In order to streamline the review of the contribution we ask you
   to ensure the following steps have been taken:
   
   ### For all changes:
   - [ ] Is there a JIRA ticket associated with this PR? Is it referenced 
        in the commit message?
   
   - [ ] Does your PR title start with **NIFI-XXXX** where XXXX is the JIRA number you are trying to resolve? Pay particular attention to the hyphen "-" character.
   
   - [ ] Has your PR been rebased against the latest commit within the target branch (typically `main`)?
   
   - [ ] Is your initial contribution a single, squashed commit? _Additional commits in response to PR reviewer feedback should be made on this branch and pushed to allow change tracking. Do not `squash` or use `--force` when pushing to allow for clean monitoring of changes._
   
   ### For code changes:
   - [ ] Have you ensured that the full suite of tests is executed via `mvn -Pcontrib-check clean install` at the root `nifi` folder?
   - [ ] Have you written or updated unit tests to verify your changes?
   - [ ] Have you verified that the full build is successful on JDK 8?
   - [ ] Have you verified that the full build is successful on JDK 11?
   - [ ] If adding new dependencies to the code, are these dependencies licensed in a way that is compatible for inclusion under [ASF 2.0](http://www.apache.org/legal/resolved.html#category-a)? 
   - [ ] If applicable, have you updated the `LICENSE` file, including the main `LICENSE` file under `nifi-assembly`?
   - [ ] If applicable, have you updated the `NOTICE` file, including the main `NOTICE` file found under `nifi-assembly`?
   - [ ] If adding new Properties, have you added `.displayName` in addition to .name (programmatic access) for each of the new properties?
   
   ### For documentation related changes:
   - [ ] Have you ensured that format looks appropriate for the output in which it is rendered?
   
   ### Note:
   Please ensure that once the PR is submitted, you check GitHub Actions CI for build issues and submit an update to your PR as soon as possible.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] markap14 commented on a change in pull request #5412: NIFI-9239: Updated Consume/Publish Kafka processors to support Exactl…

Posted by GitBox <gi...@apache.org>.
markap14 commented on a change in pull request #5412:
URL: https://github.com/apache/nifi/pull/5412#discussion_r717719235



##########
File path: nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/ExecuteStateless.java
##########
@@ -0,0 +1,1202 @@
+/*
+ * 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 com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.module.jaxb.JaxbAnnotationIntrospector;
+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.ComponentLog;
+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.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.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 javax.net.ssl.SSLContext;
+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 {
+    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");
+    static final AllowableValue SPEC_FROM_REGISTRY = new AllowableValue("Use NiFi Registry", "Use NiFi Registry", "Dataflow to run is stored in NiFi Registry");
+
+    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.");
+    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.");
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    static final PropertyDescriptor DATAFLOW_TIMEOUT = new Builder()
+        .name("Dataflow Timeout")
+        .displayName("Dataflow Timeout")
+        .description("Specifies the maximum amount of time for an invocation of the stateless flow to complete. If the flow does not complete within this amount of time, the incoming FlowFile, if " +
+            "any, will be routed to the timeout relationship. In any case, the dataflow will be canceled and the invocation will end")
+        .required(true)
+        .addValidator(TIME_PERIOD_VALIDATOR)
+        .expressionLanguageSupported(FLOWFILE_ATTRIBUTES)
+        .defaultValue("60 sec")
+        .build();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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 temporal 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();
+
+    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 DataflowRetrieval rawRetrieval;
+        if (specificationStrategy.equalsIgnoreCase(SPEC_FROM_FILE.getValue())) {
+            rawRetrieval = new FileSystemDataflowRetrieval();
+        } else {
+            rawRetrieval = new RegistryDataflowRetrieval(getLogger());
+        }
+
+        final DataflowRetrieval cachedRetrieval = new CachingDataflowRetrieval(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 {
+                final LogLevel logLevel = LogLevel.valueOf(bulletin.getLevel());
+                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 ignored) {
+                // There should be no malformed bulletin, but just in case the bulletin level, etc. ends up being null, catch the exception and ignore it,
+                // since there's not much we can do about it.
+            }
+        }
+    }
+
+    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, "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) {

Review comment:
       Yeah while testing & debugging, I saw things in the logs indicating Processor ID, etc. and I couldn't search based on that. So this will be helpful when seeing something in the logs and wanting to see how that processor is used in the flow.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] markap14 commented on a change in pull request #5412: NIFI-9239: Updated Consume/Publish Kafka processors to support Exactl…

Posted by GitBox <gi...@apache.org>.
markap14 commented on a change in pull request #5412:
URL: https://github.com/apache/nifi/pull/5412#discussion_r716938698



##########
File path: nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/ExecuteStateless.java
##########
@@ -0,0 +1,1202 @@
+/*
+ * 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 com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.module.jaxb.JaxbAnnotationIntrospector;
+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.ComponentLog;
+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.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.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 javax.net.ssl.SSLContext;
+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 {
+    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");
+    static final AllowableValue SPEC_FROM_REGISTRY = new AllowableValue("Use NiFi Registry", "Use NiFi Registry", "Dataflow to run is stored in NiFi Registry");
+
+    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.");
+    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.");
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    static final PropertyDescriptor DATAFLOW_TIMEOUT = new Builder()
+        .name("Dataflow Timeout")
+        .displayName("Dataflow Timeout")
+        .description("Specifies the maximum amount of time for an invocation of the stateless flow to complete. If the flow does not complete within this amount of time, the incoming FlowFile, if " +
+            "any, will be routed to the timeout relationship. In any case, the dataflow will be canceled and the invocation will end")
+        .required(true)
+        .addValidator(TIME_PERIOD_VALIDATOR)
+        .expressionLanguageSupported(FLOWFILE_ATTRIBUTES)
+        .defaultValue("60 sec")
+        .build();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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 temporal 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();
+
+    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 DataflowRetrieval rawRetrieval;
+        if (specificationStrategy.equalsIgnoreCase(SPEC_FROM_FILE.getValue())) {
+            rawRetrieval = new FileSystemDataflowRetrieval();
+        } else {
+            rawRetrieval = new RegistryDataflowRetrieval(getLogger());
+        }
+
+        final DataflowRetrieval cachedRetrieval = new CachingDataflowRetrieval(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) {

Review comment:
       IOException is the only checked Exception. That's all that I was intending to catch here. 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] exceptionfactory commented on a change in pull request #5412: NIFI-9239: Updated Consume/Publish Kafka processors to support Exactl…

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on a change in pull request #5412:
URL: https://github.com/apache/nifi/pull/5412#discussion_r719601928



##########
File path: nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/retrieval/CachingDataflowRetrieval.java
##########
@@ -0,0 +1,92 @@
+/*
+ * 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;
+
+public class CachingDataflowRetrieval implements DataflowRetrieval {
+    private final String processorId;
+    private final ComponentLog logger;
+    private final DataflowRetrieval delegate;
+    private final ObjectMapper objectMapper;
+
+
+    public CachingDataflowRetrieval(final String processorId, final ComponentLog logger, final DataflowRetrieval 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 {

Review comment:
       Thanks for confirming.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] exceptionfactory commented on a change in pull request #5412: NIFI-9239: Updated Consume/Publish Kafka processors to support Exactl…

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on a change in pull request #5412:
URL: https://github.com/apache/nifi/pull/5412#discussion_r719488902



##########
File path: nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/retrieval/CachingDataflowRetrieval.java
##########
@@ -0,0 +1,92 @@
+/*
+ * 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;
+
+public class CachingDataflowRetrieval implements DataflowRetrieval {
+    private final String processorId;
+    private final ComponentLog logger;
+    private final DataflowRetrieval delegate;
+    private final ObjectMapper objectMapper;
+
+
+    public CachingDataflowRetrieval(final String processorId, final ComponentLog logger, final DataflowRetrieval 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 from Registry. Will restore Flow Snapshot from cached version at {}", cacheFile.getAbsolutePath(), e);

Review comment:
       This message implies retrieval from Registry, but this caching class does not appear specific to Registry.

##########
File path: nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/retrieval/CachingDataflowRetrieval.java
##########
@@ -0,0 +1,92 @@
+/*
+ * 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;
+
+public class CachingDataflowRetrieval implements DataflowRetrieval {
+    private final String processorId;
+    private final ComponentLog logger;
+    private final DataflowRetrieval delegate;
+    private final ObjectMapper objectMapper;
+
+
+    public CachingDataflowRetrieval(final String processorId, final ComponentLog logger, final DataflowRetrieval 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 {

Review comment:
       Are there potential concurrency concerns with this method? Could multiple threads running `ExecuteStateless` call this method causing multiple attempts to write the cache file?

##########
File path: nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/retrieval/CachingDataflowRetrieval.java
##########
@@ -0,0 +1,92 @@
+/*
+ * 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;
+
+public class CachingDataflowRetrieval implements DataflowRetrieval {
+    private final String processorId;
+    private final ComponentLog logger;
+    private final DataflowRetrieval delegate;
+    private final ObjectMapper objectMapper;
+
+
+    public CachingDataflowRetrieval(final String processorId, final ComponentLog logger, final DataflowRetrieval 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 from Registry. 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 from Registry but cannot create directory {} in order to cache the dataflow. " +

Review comment:
       Similar question regarding mention of Registry retrieval.

##########
File path: nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/retrieval/DataflowRetrieval.java
##########
@@ -0,0 +1,27 @@
+/*
+ * 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.processor.ProcessContext;
+import org.apache.nifi.registry.flow.VersionedFlowSnapshot;
+
+import java.io.IOException;
+
+public interface DataflowRetrieval {

Review comment:
       What do you think about naming this something like `DataflowProvider` or `DataflowRetriever`?

##########
File path: nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/retrieval/CachingDataflowRetrieval.java
##########
@@ -0,0 +1,92 @@
+/*
+ * 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;
+

Review comment:
       A brief summary of the general caching approach would be helpful. It appears that a `VersionedFlowSnapshot` will be retrieved from the delegate and cached on every invocation, but the cache is only used on failures?

##########
File path: nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/retrieval/FileSystemDataflowRetrieval.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 FileSystemDataflowRetrieval implements DataflowRetrieval {
+    @Override
+    public VersionedFlowSnapshot retrieveDataflowContents(final ProcessContext context) throws IOException {
+        final ObjectMapper objectMapper = new ObjectMapper();
+        objectMapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);

Review comment:
       The instantiation of `ObjectMapper` in the caching implementation includes the `JaxbAnnotationIntrospector`, should that be included here? Perhaps instantiation should be moved to a shared location.

##########
File path: nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor-tests/pom.xml
##########
@@ -0,0 +1,174 @@
+<?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>
+
+
+
+

Review comment:
       Looks like some unnecessary newlines. Given the direct use of `ObjectMapper` in multiple places, recommend adding `jackson-databind` as an explicit dependency as opposed to depending on it transitively through `nifi-registry-client`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] gresockj commented on a change in pull request #5412: NIFI-9239: Updated Consume/Publish Kafka processors to support Exactl…

Posted by GitBox <gi...@apache.org>.
gresockj commented on a change in pull request #5412:
URL: https://github.com/apache/nifi/pull/5412#discussion_r718102474



##########
File path: nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/ExecuteStateless.java
##########
@@ -0,0 +1,1202 @@
+/*
+ * 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 com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.module.jaxb.JaxbAnnotationIntrospector;
+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.ComponentLog;
+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.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.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 javax.net.ssl.SSLContext;
+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 {
+    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");
+    static final AllowableValue SPEC_FROM_REGISTRY = new AllowableValue("Use NiFi Registry", "Use NiFi Registry", "Dataflow to run is stored in NiFi Registry");
+
+    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.");
+    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.");
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    static final PropertyDescriptor DATAFLOW_TIMEOUT = new Builder()
+        .name("Dataflow Timeout")
+        .displayName("Dataflow Timeout")
+        .description("Specifies the maximum amount of time for an invocation of the stateless flow to complete. If the flow does not complete within this amount of time, the incoming FlowFile, if " +
+            "any, will be routed to the timeout relationship. In any case, the dataflow will be canceled and the invocation will end")
+        .required(true)
+        .addValidator(TIME_PERIOD_VALIDATOR)
+        .expressionLanguageSupported(FLOWFILE_ATTRIBUTES)
+        .defaultValue("60 sec")
+        .build();
+
+    static final PropertyDescriptor LIB_DIRECTORY = new Builder()
+        .name("NAR Directory")
+        .displayName("NAR Directory")
+        .description("The directory to retrieve NAR's from")

Review comment:
       (this was missed in the latest commit)




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] gresockj commented on a change in pull request #5412: NIFI-9239: Updated Consume/Publish Kafka processors to support Exactl…

Posted by GitBox <gi...@apache.org>.
gresockj commented on a change in pull request #5412:
URL: https://github.com/apache/nifi/pull/5412#discussion_r716179469



##########
File path: 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,18 @@
             .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. Typically, we want this value set to true " +

Review comment:
       I'd rephrase to avoid "we" here.  Perhaps, "In the typical use case, this value is set to true..."
   
   I think we should also be more direct in tying this to Exactly Once in Stateless.  How about:
   
   "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."

##########
File path: nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaRecord_2_6.java
##########
@@ -536,6 +550,81 @@ public void process(final InputStream in) throws IOException {
         }
     }
 
+    private List<FlowFile> pollFlowFiles(final ProcessSession session) {
+        final List<FlowFile> initialFlowFiles = session.get(FlowFileFilters.newSizeBasedFilter(1, DataUnit.MB, 500));

Review comment:
       Is there any value in making this a processor property with a default of 500MB?  Otherwise, let's at least make this a constant.

##########
File path: nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaRecord_2_6.java
##########
@@ -536,6 +550,81 @@ public void process(final InputStream in) throws IOException {
         }
     }
 
+    private 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 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);

Review comment:
       Constant?

##########
File path: nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafka_2_6.java
##########
@@ -493,6 +517,82 @@ public void process(final InputStream rawIn) throws IOException {
         }
     }
 
+    private List<FlowFile> pollFlowFiles(final ProcessContext context, final ProcessSession session) {

Review comment:
       Is there any of this logic that can be reused in `PublishKafkaRecord_2_6"?  (this and the next few methods)

##########
File path: nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/ExecuteStateless.java
##########
@@ -0,0 +1,1202 @@
+/*
+ * 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 com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.module.jaxb.JaxbAnnotationIntrospector;
+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.ComponentLog;
+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.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.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 javax.net.ssl.SSLContext;
+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 {
+    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");
+    static final AllowableValue SPEC_FROM_REGISTRY = new AllowableValue("Use NiFi Registry", "Use NiFi Registry", "Dataflow to run is stored in NiFi Registry");
+
+    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.");
+    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.");
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    static final PropertyDescriptor DATAFLOW_TIMEOUT = new Builder()
+        .name("Dataflow Timeout")
+        .displayName("Dataflow Timeout")
+        .description("Specifies the maximum amount of time for an invocation of the stateless flow to complete. If the flow does not complete within this amount of time, the incoming FlowFile, if " +
+            "any, will be routed to the timeout relationship. In any case, the dataflow will be canceled and the invocation will end")
+        .required(true)
+        .addValidator(TIME_PERIOD_VALIDATOR)
+        .expressionLanguageSupported(FLOWFILE_ATTRIBUTES)
+        .defaultValue("60 sec")
+        .build();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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 temporal 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();
+
+    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.")

Review comment:
       Would it be worth defining a second failure relationship, perhaps "rejected" for this purpose?  This would help distinguish flowfiles that weren't processed from failures coming out of the data flow.  I suppose this case could be discerned from the lack of `failure.port.name` attribute, however.  Just wanted to pose the question.

##########
File path: 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,357 @@
+<!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 if using an external URL, if the Processor is stopped and started (or NiFi is restarted), it is possible that

Review comment:
       "if using an external URL or NiFi Registry"

##########
File path: nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/ExecuteStateless.java
##########
@@ -0,0 +1,1202 @@
+/*
+ * 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 com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.module.jaxb.JaxbAnnotationIntrospector;
+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.ComponentLog;
+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.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.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 javax.net.ssl.SSLContext;
+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 {
+    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");
+    static final AllowableValue SPEC_FROM_REGISTRY = new AllowableValue("Use NiFi Registry", "Use NiFi Registry", "Dataflow to run is stored in NiFi Registry");
+
+    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.");
+    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.");
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    static final PropertyDescriptor DATAFLOW_TIMEOUT = new Builder()
+        .name("Dataflow Timeout")
+        .displayName("Dataflow Timeout")
+        .description("Specifies the maximum amount of time for an invocation of the stateless flow to complete. If the flow does not complete within this amount of time, the incoming FlowFile, if " +
+            "any, will be routed to the timeout relationship. In any case, the dataflow will be canceled and the invocation will end")
+        .required(true)
+        .addValidator(TIME_PERIOD_VALIDATOR)
+        .expressionLanguageSupported(FLOWFILE_ATTRIBUTES)
+        .defaultValue("60 sec")
+        .build();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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 temporal 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();
+
+    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 DataflowRetrieval rawRetrieval;
+        if (specificationStrategy.equalsIgnoreCase(SPEC_FROM_FILE.getValue())) {
+            rawRetrieval = new FileSystemDataflowRetrieval();
+        } else {
+            rawRetrieval = new RegistryDataflowRetrieval(getLogger());
+        }
+
+        final DataflowRetrieval cachedRetrieval = new CachingDataflowRetrieval(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 {
+                final LogLevel logLevel = LogLevel.valueOf(bulletin.getLevel());
+                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 ignored) {
+                // There should be no malformed bulletin, but just in case the bulletin level, etc. ends up being null, catch the exception and ignore it,
+                // since there's not much we can do about it.
+            }
+        }
+    }
+
+    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, "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) {

Review comment:
       This is a nice touch.  Great idea to make the stateless flow searchable!  It looks nice in action.

##########
File path: nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/ExecuteStateless.java
##########
@@ -0,0 +1,1202 @@
+/*
+ * 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 com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.module.jaxb.JaxbAnnotationIntrospector;
+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.ComponentLog;
+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.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.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 javax.net.ssl.SSLContext;
+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 {
+    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");
+    static final AllowableValue SPEC_FROM_REGISTRY = new AllowableValue("Use NiFi Registry", "Use NiFi Registry", "Dataflow to run is stored in NiFi Registry");
+
+    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.");
+    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.");
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    static final PropertyDescriptor DATAFLOW_TIMEOUT = new Builder()
+        .name("Dataflow Timeout")
+        .displayName("Dataflow Timeout")
+        .description("Specifies the maximum amount of time for an invocation of the stateless flow to complete. If the flow does not complete within this amount of time, the incoming FlowFile, if " +
+            "any, will be routed to the timeout relationship. In any case, the dataflow will be canceled and the invocation will end")
+        .required(true)
+        .addValidator(TIME_PERIOD_VALIDATOR)
+        .expressionLanguageSupported(FLOWFILE_ATTRIBUTES)
+        .defaultValue("60 sec")
+        .build();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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 temporal 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();
+
+    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 DataflowRetrieval rawRetrieval;
+        if (specificationStrategy.equalsIgnoreCase(SPEC_FROM_FILE.getValue())) {
+            rawRetrieval = new FileSystemDataflowRetrieval();
+        } else {
+            rawRetrieval = new RegistryDataflowRetrieval(getLogger());
+        }
+
+        final DataflowRetrieval cachedRetrieval = new CachingDataflowRetrieval(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 {
+                final LogLevel logLevel = LogLevel.valueOf(bulletin.getLevel());
+                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 ignored) {
+                // There should be no malformed bulletin, but just in case the bulletin level, etc. ends up being null, catch the exception and ignore it,
+                // since there's not much we can do about it.
+            }
+        }
+    }
+
+    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, "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 interface DataflowRetrieval {
+        VersionedFlowSnapshot retrieveDataflowContents(ProcessContext context) throws IOException;
+    }
+
+    private static class FileSystemDataflowRetrieval implements DataflowRetrieval {
+        @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(DATAFLOW_FILE).asResource().read()) {
+                final VersionedFlowSnapshot versionedFlowSnapshot = objectMapper.readValue(in, VersionedFlowSnapshot.class);
+                return versionedFlowSnapshot;
+            }
+        }
+    }
+
+    private static class CachingDataflowRetrieval implements DataflowRetrieval {
+        private final String processorId;
+        private final ComponentLog logger;
+        private final DataflowRetrieval delegate;
+
+        public CachingDataflowRetrieval(final String processorId, final ComponentLog logger, final DataflowRetrieval delegate) {
+            this.processorId = processorId;
+            this.logger = logger;
+            this.delegate = delegate;
+        }
+
+        @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 from Registry. 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 from Registry but cannot create directory {} in order to cache the dataflow. " +
+                    "Upon restart, processor will not be able to function unless Registry is available", cacheFile);
+                return;
+            }
+
+            try {
+                final ObjectMapper objectMapper = new ObjectMapper();

Review comment:
       Could the `objectMapper` be moved to a private field so you don't have to configure it in two places?

##########
File path: nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/ExecuteStateless.java
##########
@@ -0,0 +1,1202 @@
+/*
+ * 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 com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.module.jaxb.JaxbAnnotationIntrospector;
+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.ComponentLog;
+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.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.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 javax.net.ssl.SSLContext;
+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 {
+    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");
+    static final AllowableValue SPEC_FROM_REGISTRY = new AllowableValue("Use NiFi Registry", "Use NiFi Registry", "Dataflow to run is stored in NiFi Registry");
+
+    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.");
+    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.");
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    static final PropertyDescriptor DATAFLOW_TIMEOUT = new Builder()
+        .name("Dataflow Timeout")
+        .displayName("Dataflow Timeout")
+        .description("Specifies the maximum amount of time for an invocation of the stateless flow to complete. If the flow does not complete within this amount of time, the incoming FlowFile, if " +
+            "any, will be routed to the timeout relationship. In any case, the dataflow will be canceled and the invocation will end")
+        .required(true)
+        .addValidator(TIME_PERIOD_VALIDATOR)
+        .expressionLanguageSupported(FLOWFILE_ATTRIBUTES)
+        .defaultValue("60 sec")
+        .build();
+
+    static final PropertyDescriptor LIB_DIRECTORY = new Builder()
+        .name("NAR Directory")
+        .displayName("NAR Directory")
+        .description("The directory to retrieve NAR's from")

Review comment:
       No apostrophe here

##########
File path: nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/ExecuteStateless.java
##########
@@ -0,0 +1,1202 @@
+/*
+ * 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 com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.module.jaxb.JaxbAnnotationIntrospector;
+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.ComponentLog;
+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.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.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 javax.net.ssl.SSLContext;
+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 {
+    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");
+    static final AllowableValue SPEC_FROM_REGISTRY = new AllowableValue("Use NiFi Registry", "Use NiFi Registry", "Dataflow to run is stored in NiFi Registry");
+
+    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.");
+    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.");
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    static final PropertyDescriptor DATAFLOW_TIMEOUT = new Builder()
+        .name("Dataflow Timeout")
+        .displayName("Dataflow Timeout")
+        .description("Specifies the maximum amount of time for an invocation of the stateless flow to complete. If the flow does not complete within this amount of time, the incoming FlowFile, if " +
+            "any, will be routed to the timeout relationship. In any case, the dataflow will be canceled and the invocation will end")

Review comment:
       Missing a period.  Also, what do you mean by "in any case"?  It seems clearer to say: "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."

##########
File path: nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/ExecuteStateless.java
##########
@@ -0,0 +1,1202 @@
+/*
+ * 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 com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.module.jaxb.JaxbAnnotationIntrospector;
+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.ComponentLog;
+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.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.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 javax.net.ssl.SSLContext;
+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 {
+    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");
+    static final AllowableValue SPEC_FROM_REGISTRY = new AllowableValue("Use NiFi Registry", "Use NiFi Registry", "Dataflow to run is stored in NiFi Registry");
+
+    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.");
+    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.");
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    static final PropertyDescriptor DATAFLOW_TIMEOUT = new Builder()
+        .name("Dataflow Timeout")
+        .displayName("Dataflow Timeout")
+        .description("Specifies the maximum amount of time for an invocation of the stateless flow to complete. If the flow does not complete within this amount of time, the incoming FlowFile, if " +
+            "any, will be routed to the timeout relationship. In any case, the dataflow will be canceled and the invocation will end")
+        .required(true)
+        .addValidator(TIME_PERIOD_VALIDATOR)
+        .expressionLanguageSupported(FLOWFILE_ATTRIBUTES)
+        .defaultValue("60 sec")
+        .build();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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 temporal 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();
+
+    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 DataflowRetrieval rawRetrieval;
+        if (specificationStrategy.equalsIgnoreCase(SPEC_FROM_FILE.getValue())) {
+            rawRetrieval = new FileSystemDataflowRetrieval();
+        } else {
+            rawRetrieval = new RegistryDataflowRetrieval(getLogger());
+        }
+
+        final DataflowRetrieval cachedRetrieval = new CachingDataflowRetrieval(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();

Review comment:
       I may be missing something, but I couldn't get any bulletins to appear on the ExecuteStateless processor when running a dataflow that produced an ERROR bulletin -- `maxBulletinId` was always -1 here during my testing.

##########
File path: nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/ExecuteStateless.java
##########
@@ -0,0 +1,1202 @@
+/*
+ * 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 com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.module.jaxb.JaxbAnnotationIntrospector;
+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.ComponentLog;
+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.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.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 javax.net.ssl.SSLContext;
+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 {
+    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");
+    static final AllowableValue SPEC_FROM_REGISTRY = new AllowableValue("Use NiFi Registry", "Use NiFi Registry", "Dataflow to run is stored in NiFi Registry");
+
+    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.");
+    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.");
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    static final PropertyDescriptor DATAFLOW_TIMEOUT = new Builder()
+        .name("Dataflow Timeout")
+        .displayName("Dataflow Timeout")
+        .description("Specifies the maximum amount of time for an invocation of the stateless flow to complete. If the flow does not complete within this amount of time, the incoming FlowFile, if " +
+            "any, will be routed to the timeout relationship. In any case, the dataflow will be canceled and the invocation will end")
+        .required(true)
+        .addValidator(TIME_PERIOD_VALIDATOR)
+        .expressionLanguageSupported(FLOWFILE_ATTRIBUTES)
+        .defaultValue("60 sec")
+        .build();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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 temporal and may be deleted at " +

Review comment:
       temporary instead of temporal

##########
File path: 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,357 @@
+<!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 if using an external URL, 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).
+</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 make

Review comment:
       Remove "make" from "can make provide"

##########
File path: nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/ExecuteStateless.java
##########
@@ -0,0 +1,1202 @@
+/*
+ * 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 com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.module.jaxb.JaxbAnnotationIntrospector;
+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.ComponentLog;
+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.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.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 javax.net.ssl.SSLContext;
+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 {
+    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");
+    static final AllowableValue SPEC_FROM_REGISTRY = new AllowableValue("Use NiFi Registry", "Use NiFi Registry", "Dataflow to run is stored in NiFi Registry");
+
+    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.");
+    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.");
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    static final PropertyDescriptor DATAFLOW_TIMEOUT = new Builder()
+        .name("Dataflow Timeout")
+        .displayName("Dataflow Timeout")
+        .description("Specifies the maximum amount of time for an invocation of the stateless flow to complete. If the flow does not complete within this amount of time, the incoming FlowFile, if " +
+            "any, will be routed to the timeout relationship. In any case, the dataflow will be canceled and the invocation will end")
+        .required(true)
+        .addValidator(TIME_PERIOD_VALIDATOR)
+        .expressionLanguageSupported(FLOWFILE_ATTRIBUTES)
+        .defaultValue("60 sec")
+        .build();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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 temporal 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();
+
+    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 DataflowRetrieval rawRetrieval;
+        if (specificationStrategy.equalsIgnoreCase(SPEC_FROM_FILE.getValue())) {
+            rawRetrieval = new FileSystemDataflowRetrieval();
+        } else {
+            rawRetrieval = new RegistryDataflowRetrieval(getLogger());
+        }
+
+        final DataflowRetrieval cachedRetrieval = new CachingDataflowRetrieval(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 {
+                final LogLevel logLevel = LogLevel.valueOf(bulletin.getLevel());
+                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 ignored) {
+                // There should be no malformed bulletin, but just in case the bulletin level, etc. ends up being null, catch the exception and ignore it,
+                // since there's not much we can do about it.
+            }
+        }
+    }
+
+    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, "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 interface DataflowRetrieval {
+        VersionedFlowSnapshot retrieveDataflowContents(ProcessContext context) throws IOException;
+    }
+
+    private static class FileSystemDataflowRetrieval implements DataflowRetrieval {
+        @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(DATAFLOW_FILE).asResource().read()) {
+                final VersionedFlowSnapshot versionedFlowSnapshot = objectMapper.readValue(in, VersionedFlowSnapshot.class);
+                return versionedFlowSnapshot;
+            }
+        }
+    }
+
+    private static class CachingDataflowRetrieval implements DataflowRetrieval {
+        private final String processorId;
+        private final ComponentLog logger;
+        private final DataflowRetrieval delegate;
+
+        public CachingDataflowRetrieval(final String processorId, final ComponentLog logger, final DataflowRetrieval delegate) {
+            this.processorId = processorId;
+            this.logger = logger;
+            this.delegate = delegate;
+        }
+
+        @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 from Registry. Will restore Flow Snapshot from cached version at {}", cacheFile.getAbsolutePath(), e);

Review comment:
       "Flow"

##########
File path: nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/ExecuteStateless.java
##########
@@ -0,0 +1,1202 @@
+/*
+ * 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 com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.module.jaxb.JaxbAnnotationIntrospector;
+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.ComponentLog;
+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.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.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 javax.net.ssl.SSLContext;
+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 {
+    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");
+    static final AllowableValue SPEC_FROM_REGISTRY = new AllowableValue("Use NiFi Registry", "Use NiFi Registry", "Dataflow to run is stored in NiFi Registry");
+
+    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.");
+    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.");
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    static final PropertyDescriptor DATAFLOW_TIMEOUT = new Builder()
+        .name("Dataflow Timeout")
+        .displayName("Dataflow Timeout")
+        .description("Specifies the maximum amount of time for an invocation of the stateless flow to complete. If the flow does not complete within this amount of time, the incoming FlowFile, if " +
+            "any, will be routed to the timeout relationship. In any case, the dataflow will be canceled and the invocation will end")
+        .required(true)
+        .addValidator(TIME_PERIOD_VALIDATOR)
+        .expressionLanguageSupported(FLOWFILE_ATTRIBUTES)
+        .defaultValue("60 sec")
+        .build();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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 temporal 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();
+
+    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 DataflowRetrieval rawRetrieval;
+        if (specificationStrategy.equalsIgnoreCase(SPEC_FROM_FILE.getValue())) {
+            rawRetrieval = new FileSystemDataflowRetrieval();
+        } else {
+            rawRetrieval = new RegistryDataflowRetrieval(getLogger());
+        }
+
+        final DataflowRetrieval cachedRetrieval = new CachingDataflowRetrieval(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 {
+                final LogLevel logLevel = LogLevel.valueOf(bulletin.getLevel());
+                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 ignored) {
+                // There should be no malformed bulletin, but just in case the bulletin level, etc. ends up being null, catch the exception and ignore it,
+                // since there's not much we can do about it.
+            }
+        }
+    }
+
+    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, "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 interface DataflowRetrieval {

Review comment:
       I actually had the same thought when initially reading through the code.  Might such a controller service be useful outside of ExecuteStateless?  Not sure if this would be overkill, but worth considering.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] markap14 commented on a change in pull request #5412: NIFI-9239: Updated Consume/Publish Kafka processors to support Exactl…

Posted by GitBox <gi...@apache.org>.
markap14 commented on a change in pull request #5412:
URL: https://github.com/apache/nifi/pull/5412#discussion_r716931947



##########
File path: nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/ExecuteStateless.java
##########
@@ -0,0 +1,1202 @@
+/*
+ * 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 com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.module.jaxb.JaxbAnnotationIntrospector;
+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.ComponentLog;
+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.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.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 javax.net.ssl.SSLContext;
+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 {
+    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");
+    static final AllowableValue SPEC_FROM_REGISTRY = new AllowableValue("Use NiFi Registry", "Use NiFi Registry", "Dataflow to run is stored in NiFi Registry");
+
+    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.");
+    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.");
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    static final PropertyDescriptor DATAFLOW_TIMEOUT = new Builder()
+        .name("Dataflow Timeout")
+        .displayName("Dataflow Timeout")
+        .description("Specifies the maximum amount of time for an invocation of the stateless flow to complete. If the flow does not complete within this amount of time, the incoming FlowFile, if " +
+            "any, will be routed to the timeout relationship. In any case, the dataflow will be canceled and the invocation will end")
+        .required(true)
+        .addValidator(TIME_PERIOD_VALIDATOR)
+        .expressionLanguageSupported(FLOWFILE_ATTRIBUTES)
+        .defaultValue("60 sec")
+        .build();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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 temporal 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();
+
+    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)

Review comment:
       I did consider that. But I found that to be very confusing and not very straight forward. It also adds complexity in that the person who is to run this component needs to know whether or not each parameter is sensitive rather just knowing which parameters need to be set. I could probably be convinced to go the route of checking if the property starts with "Sensitive" but I think I lean a bit more toward this approach.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] markap14 commented on a change in pull request #5412: NIFI-9239: Updated Consume/Publish Kafka processors to support Exactl…

Posted by GitBox <gi...@apache.org>.
markap14 commented on a change in pull request #5412:
URL: https://github.com/apache/nifi/pull/5412#discussion_r716998793



##########
File path: nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/ExecuteStateless.java
##########
@@ -0,0 +1,1202 @@
+/*
+ * 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 com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.module.jaxb.JaxbAnnotationIntrospector;
+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.ComponentLog;
+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.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.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 javax.net.ssl.SSLContext;
+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 {
+    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");
+    static final AllowableValue SPEC_FROM_REGISTRY = new AllowableValue("Use NiFi Registry", "Use NiFi Registry", "Dataflow to run is stored in NiFi Registry");
+
+    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.");
+    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.");
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    static final PropertyDescriptor DATAFLOW_TIMEOUT = new Builder()
+        .name("Dataflow Timeout")
+        .displayName("Dataflow Timeout")
+        .description("Specifies the maximum amount of time for an invocation of the stateless flow to complete. If the flow does not complete within this amount of time, the incoming FlowFile, if " +
+            "any, will be routed to the timeout relationship. In any case, the dataflow will be canceled and the invocation will end")
+        .required(true)
+        .addValidator(TIME_PERIOD_VALIDATOR)
+        .expressionLanguageSupported(FLOWFILE_ATTRIBUTES)
+        .defaultValue("60 sec")
+        .build();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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 temporal 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();
+
+    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 DataflowRetrieval rawRetrieval;
+        if (specificationStrategy.equalsIgnoreCase(SPEC_FROM_FILE.getValue())) {
+            rawRetrieval = new FileSystemDataflowRetrieval();
+        } else {
+            rawRetrieval = new RegistryDataflowRetrieval(getLogger());
+        }
+
+        final DataflowRetrieval cachedRetrieval = new CachingDataflowRetrieval(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 {
+                final LogLevel logLevel = LogLevel.valueOf(bulletin.getLevel());
+                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 ignored) {
+                // There should be no malformed bulletin, but just in case the bulletin level, etc. ends up being null, catch the exception and ignore it,
+                // since there's not much we can do about it.
+            }
+        }
+    }
+
+    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, "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 interface DataflowRetrieval {

Review comment:
       I would avoid controller services in this situation. The advantage to using a controller service would be that you could configure it once and reuse it many times for many instances of ExecuteStateless. Except that for registry you'd still need a different bucket, flow, version every time. All you'd really be able to reuse is the timeout, url, and maybe SSL Context. So the benefits of a controller service here, IMO, are outweighed but the complexity of having to configure that service.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] exceptionfactory commented on a change in pull request #5412: NIFI-9239: Updated Consume/Publish Kafka processors to support Exactl…

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on a change in pull request #5412:
URL: https://github.com/apache/nifi/pull/5412#discussion_r716944946



##########
File path: nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/ExecuteStateless.java
##########
@@ -0,0 +1,1202 @@
+/*
+ * 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 com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.module.jaxb.JaxbAnnotationIntrospector;
+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.ComponentLog;
+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.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.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 javax.net.ssl.SSLContext;
+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 {
+    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");
+    static final AllowableValue SPEC_FROM_REGISTRY = new AllowableValue("Use NiFi Registry", "Use NiFi Registry", "Dataflow to run is stored in NiFi Registry");
+
+    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.");
+    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.");
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    static final PropertyDescriptor DATAFLOW_TIMEOUT = new Builder()
+        .name("Dataflow Timeout")
+        .displayName("Dataflow Timeout")
+        .description("Specifies the maximum amount of time for an invocation of the stateless flow to complete. If the flow does not complete within this amount of time, the incoming FlowFile, if " +
+            "any, will be routed to the timeout relationship. In any case, the dataflow will be canceled and the invocation will end")
+        .required(true)
+        .addValidator(TIME_PERIOD_VALIDATOR)
+        .expressionLanguageSupported(FLOWFILE_ATTRIBUTES)
+        .defaultValue("60 sec")
+        .build();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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 temporal 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();
+
+    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 DataflowRetrieval rawRetrieval;
+        if (specificationStrategy.equalsIgnoreCase(SPEC_FROM_FILE.getValue())) {
+            rawRetrieval = new FileSystemDataflowRetrieval();
+        } else {
+            rawRetrieval = new RegistryDataflowRetrieval(getLogger());
+        }
+
+        final DataflowRetrieval cachedRetrieval = new CachingDataflowRetrieval(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 {
+                final LogLevel logLevel = LogLevel.valueOf(bulletin.getLevel());
+                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 ignored) {
+                // There should be no malformed bulletin, but just in case the bulletin level, etc. ends up being null, catch the exception and ignore it,
+                // since there's not much we can do about it.
+            }
+        }
+    }
+
+    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, "flowfile-content");

Review comment:
       Thanks for the clarification, that makes sense!




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] markap14 commented on a change in pull request #5412: NIFI-9239: Updated Consume/Publish Kafka processors to support Exactl…

Posted by GitBox <gi...@apache.org>.
markap14 commented on a change in pull request #5412:
URL: https://github.com/apache/nifi/pull/5412#discussion_r719603059



##########
File path: nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor-tests/pom.xml
##########
@@ -0,0 +1,174 @@
+<?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>
+
+
+
+

Review comment:
       We don't use `ObjectMapper` here, in the processor-tests. I did add a dependency on `jackson-databind` in the processor module.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] gresockj commented on a change in pull request #5412: NIFI-9239: Updated Consume/Publish Kafka processors to support Exactl…

Posted by GitBox <gi...@apache.org>.
gresockj commented on a change in pull request #5412:
URL: https://github.com/apache/nifi/pull/5412#discussion_r718102474



##########
File path: nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/ExecuteStateless.java
##########
@@ -0,0 +1,1202 @@
+/*
+ * 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 com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.module.jaxb.JaxbAnnotationIntrospector;
+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.ComponentLog;
+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.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.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 javax.net.ssl.SSLContext;
+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 {
+    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");
+    static final AllowableValue SPEC_FROM_REGISTRY = new AllowableValue("Use NiFi Registry", "Use NiFi Registry", "Dataflow to run is stored in NiFi Registry");
+
+    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.");
+    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.");
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    static final PropertyDescriptor DATAFLOW_TIMEOUT = new Builder()
+        .name("Dataflow Timeout")
+        .displayName("Dataflow Timeout")
+        .description("Specifies the maximum amount of time for an invocation of the stateless flow to complete. If the flow does not complete within this amount of time, the incoming FlowFile, if " +
+            "any, will be routed to the timeout relationship. In any case, the dataflow will be canceled and the invocation will end")
+        .required(true)
+        .addValidator(TIME_PERIOD_VALIDATOR)
+        .expressionLanguageSupported(FLOWFILE_ATTRIBUTES)
+        .defaultValue("60 sec")
+        .build();
+
+    static final PropertyDescriptor LIB_DIRECTORY = new Builder()
+        .name("NAR Directory")
+        .displayName("NAR Directory")
+        .description("The directory to retrieve NAR's from")

Review comment:
       (this was missed in the latest commit)




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] markap14 commented on a change in pull request #5412: NIFI-9239: Updated Consume/Publish Kafka processors to support Exactl…

Posted by GitBox <gi...@apache.org>.
markap14 commented on a change in pull request #5412:
URL: https://github.com/apache/nifi/pull/5412#discussion_r717719817



##########
File path: nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/ExecuteStateless.java
##########
@@ -0,0 +1,1202 @@
+/*
+ * 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 com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.module.jaxb.JaxbAnnotationIntrospector;
+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.ComponentLog;
+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.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.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 javax.net.ssl.SSLContext;
+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 {
+    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");
+    static final AllowableValue SPEC_FROM_REGISTRY = new AllowableValue("Use NiFi Registry", "Use NiFi Registry", "Dataflow to run is stored in NiFi Registry");
+
+    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.");
+    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.");
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    static final PropertyDescriptor DATAFLOW_TIMEOUT = new Builder()
+        .name("Dataflow Timeout")
+        .displayName("Dataflow Timeout")
+        .description("Specifies the maximum amount of time for an invocation of the stateless flow to complete. If the flow does not complete within this amount of time, the incoming FlowFile, if " +
+            "any, will be routed to the timeout relationship. In any case, the dataflow will be canceled and the invocation will end")
+        .required(true)
+        .addValidator(TIME_PERIOD_VALIDATOR)
+        .expressionLanguageSupported(FLOWFILE_ATTRIBUTES)
+        .defaultValue("60 sec")
+        .build();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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 temporal 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();
+
+    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 DataflowRetrieval rawRetrieval;
+        if (specificationStrategy.equalsIgnoreCase(SPEC_FROM_FILE.getValue())) {
+            rawRetrieval = new FileSystemDataflowRetrieval();
+        } else {
+            rawRetrieval = new RegistryDataflowRetrieval(getLogger());
+        }
+
+        final DataflowRetrieval cachedRetrieval = new CachingDataflowRetrieval(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();

Review comment:
       Thanks. You're right. Those were not properly surfacing. I had to make a few adjustments.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] exceptionfactory commented on a change in pull request #5412: NIFI-9239: Updated Consume/Publish Kafka processors to support Exactl…

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on a change in pull request #5412:
URL: https://github.com/apache/nifi/pull/5412#discussion_r719626419



##########
File path: nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor-tests/pom.xml
##########
@@ -0,0 +1,174 @@
+<?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>
+
+
+
+

Review comment:
       Sorry for the confusion, I mixed up the context of this Maven configuration.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] markap14 commented on a change in pull request #5412: NIFI-9239: Updated Consume/Publish Kafka processors to support Exactl…

Posted by GitBox <gi...@apache.org>.
markap14 commented on a change in pull request #5412:
URL: https://github.com/apache/nifi/pull/5412#discussion_r716941477



##########
File path: nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/ExecuteStateless.java
##########
@@ -0,0 +1,1202 @@
+/*
+ * 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 com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.module.jaxb.JaxbAnnotationIntrospector;
+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.ComponentLog;
+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.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.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 javax.net.ssl.SSLContext;
+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 {
+    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");
+    static final AllowableValue SPEC_FROM_REGISTRY = new AllowableValue("Use NiFi Registry", "Use NiFi Registry", "Dataflow to run is stored in NiFi Registry");
+
+    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.");
+    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.");
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    static final PropertyDescriptor DATAFLOW_TIMEOUT = new Builder()
+        .name("Dataflow Timeout")
+        .displayName("Dataflow Timeout")
+        .description("Specifies the maximum amount of time for an invocation of the stateless flow to complete. If the flow does not complete within this amount of time, the incoming FlowFile, if " +
+            "any, will be routed to the timeout relationship. In any case, the dataflow will be canceled and the invocation will end")
+        .required(true)
+        .addValidator(TIME_PERIOD_VALIDATOR)
+        .expressionLanguageSupported(FLOWFILE_ATTRIBUTES)
+        .defaultValue("60 sec")
+        .build();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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 temporal 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();
+
+    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 DataflowRetrieval rawRetrieval;
+        if (specificationStrategy.equalsIgnoreCase(SPEC_FROM_FILE.getValue())) {
+            rawRetrieval = new FileSystemDataflowRetrieval();
+        } else {
+            rawRetrieval = new RegistryDataflowRetrieval(getLogger());
+        }
+
+        final DataflowRetrieval cachedRetrieval = new CachingDataflowRetrieval(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 {
+                final LogLevel logLevel = LogLevel.valueOf(bulletin.getLevel());
+                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 ignored) {
+                // There should be no malformed bulletin, but just in case the bulletin level, etc. ends up being null, catch the exception and ignore it,
+                // since there's not much we can do about it.
+            }
+        }
+    }
+
+    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, "flowfile-content");

Review comment:
       @exceptionfactory multiple invocations of the same processor should use the same directory. Invocations from different processors should not. This is why the next line here create a new File that is a child of 'flowfile-content' and whose name is the identifier of the Processor. This allows us to separate out content repos per processor - and even per thread, actually, as multiple concurrent threads cannot be writing to the same content repo. I could add in  'execute-stateless' to the filename.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] exceptionfactory commented on a change in pull request #5412: NIFI-9239: Updated Consume/Publish Kafka processors to support Exactl…

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on a change in pull request #5412:
URL: https://github.com/apache/nifi/pull/5412#discussion_r717008328



##########
File path: nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/ExecuteStateless.java
##########
@@ -0,0 +1,1202 @@
+/*
+ * 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 com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.module.jaxb.JaxbAnnotationIntrospector;
+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.ComponentLog;
+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.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.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 javax.net.ssl.SSLContext;
+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 {
+    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");
+    static final AllowableValue SPEC_FROM_REGISTRY = new AllowableValue("Use NiFi Registry", "Use NiFi Registry", "Dataflow to run is stored in NiFi Registry");
+
+    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.");
+    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.");
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    static final PropertyDescriptor DATAFLOW_TIMEOUT = new Builder()
+        .name("Dataflow Timeout")
+        .displayName("Dataflow Timeout")
+        .description("Specifies the maximum amount of time for an invocation of the stateless flow to complete. If the flow does not complete within this amount of time, the incoming FlowFile, if " +
+            "any, will be routed to the timeout relationship. In any case, the dataflow will be canceled and the invocation will end")
+        .required(true)
+        .addValidator(TIME_PERIOD_VALIDATOR)
+        .expressionLanguageSupported(FLOWFILE_ATTRIBUTES)
+        .defaultValue("60 sec")
+        .build();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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 temporal 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();
+
+    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 DataflowRetrieval rawRetrieval;
+        if (specificationStrategy.equalsIgnoreCase(SPEC_FROM_FILE.getValue())) {
+            rawRetrieval = new FileSystemDataflowRetrieval();
+        } else {
+            rawRetrieval = new RegistryDataflowRetrieval(getLogger());
+        }
+
+        final DataflowRetrieval cachedRetrieval = new CachingDataflowRetrieval(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 {
+                final LogLevel logLevel = LogLevel.valueOf(bulletin.getLevel());
+                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 ignored) {
+                // There should be no malformed bulletin, but just in case the bulletin level, etc. ends up being null, catch the exception and ignore it,
+                // since there's not much we can do about it.
+            }
+        }
+    }
+
+    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, "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 interface DataflowRetrieval {

Review comment:
       Could the the `Dataflow File/URL` property be repurposed to parameterize the Bucket, Flow, and Version properties? I understand the tradeoff of complexity, but the current implementation requires a number of properties that are specific to Registry.  The `dependsOn` element helps keep this more manageable, but it does add a good bit of code complexity.  It seems like another potential implementation could be direct retrieval from a Git repository or web service. If those additional implementations seem unlikely, or would not introduce too much complication to the current approach, then a Controller Service seems unnecessary.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] asfgit closed pull request #5412: NIFI-9239: Updated Consume/Publish Kafka processors to support Exactl…

Posted by GitBox <gi...@apache.org>.
asfgit closed pull request #5412:
URL: https://github.com/apache/nifi/pull/5412


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] markap14 commented on a change in pull request #5412: NIFI-9239: Updated Consume/Publish Kafka processors to support Exactl…

Posted by GitBox <gi...@apache.org>.
markap14 commented on a change in pull request #5412:
URL: https://github.com/apache/nifi/pull/5412#discussion_r719598198



##########
File path: nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/retrieval/CachingDataflowRetrieval.java
##########
@@ -0,0 +1,92 @@
+/*
+ * 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;
+
+public class CachingDataflowRetrieval implements DataflowRetrieval {
+    private final String processorId;
+    private final ComponentLog logger;
+    private final DataflowRetrieval delegate;
+    private final ObjectMapper objectMapper;
+
+
+    public CachingDataflowRetrieval(final String processorId, final ComponentLog logger, final DataflowRetrieval 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 from Registry. Will restore Flow Snapshot from cached version at {}", cacheFile.getAbsolutePath(), e);

Review comment:
       Good catch.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] markap14 commented on pull request #5412: NIFI-9239: Updated Consume/Publish Kafka processors to support Exactl…

Posted by GitBox <gi...@apache.org>.
markap14 commented on pull request #5412:
URL: https://github.com/apache/nifi/pull/5412#issuecomment-929358954


   @gresockj @exceptionfactory  huge thanks for all of the very thoughtful and thorough feedback! I will push a new commit momentarily that should address all concerns. And I definitely agree @exceptionfactory that there are a lot more things that should be unit tested. I intended to write a bunch more tests and then it totally slipped my mind. So the next commit will include new tests, mostly around error conditions, such as timeouts, processors that throw exceptions, failure ports, etc. Thanks!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] exceptionfactory commented on a change in pull request #5412: NIFI-9239: Updated Consume/Publish Kafka processors to support Exactl…

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on a change in pull request #5412:
URL: https://github.com/apache/nifi/pull/5412#discussion_r716772257



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarClassLoaders.java
##########
@@ -472,10 +472,16 @@ private ClassLoader createBundleClassLoader(final BundleDetails bundleDetail, fi
      * @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: " + narDirectory.getAbsolutePath() + " as class loader " + narClassLoader);
+        } else {
+            logger.debug("Loaded NAR file: " + narDirectory.getAbsolutePath() + " as class loader " + narClassLoader);

Review comment:
       Recommend using placeholders:
   ```suggestion
               logger.debug("Loaded NAR {} with {}", narDirectory.getAbsolutePath(), narClassLoader);
   ```

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarClassLoaders.java
##########
@@ -140,8 +140,8 @@ public void init(final ClassLoader rootClassloader,
     /**
      * 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)

Review comment:
       The `logDetails` boolean flag is not completely clear as it appears up being interpreted as `info` versus `debug`. What do you think about renaming it to `logInfoEnabled` or something similar?

##########
File path: nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/pom.xml
##########
@@ -0,0 +1,66 @@
+<?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>

Review comment:
       What do you think about adding `jackson-databind` as an explicit dependency since `ExecuteStateless` includes direct usage of the Jackson `ObjectMapper`?

##########
File path: nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/ExecuteStateless.java
##########
@@ -0,0 +1,1202 @@
+/*
+ * 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 com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.module.jaxb.JaxbAnnotationIntrospector;
+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.ComponentLog;
+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.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.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 javax.net.ssl.SSLContext;
+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 {
+    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");
+    static final AllowableValue SPEC_FROM_REGISTRY = new AllowableValue("Use NiFi Registry", "Use NiFi Registry", "Dataflow to run is stored in NiFi Registry");
+
+    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.");
+    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.");
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    static final PropertyDescriptor DATAFLOW_TIMEOUT = new Builder()
+        .name("Dataflow Timeout")
+        .displayName("Dataflow Timeout")
+        .description("Specifies the maximum amount of time for an invocation of the stateless flow to complete. If the flow does not complete within this amount of time, the incoming FlowFile, if " +
+            "any, will be routed to the timeout relationship. In any case, the dataflow will be canceled and the invocation will end")
+        .required(true)
+        .addValidator(TIME_PERIOD_VALIDATOR)
+        .expressionLanguageSupported(FLOWFILE_ATTRIBUTES)
+        .defaultValue("60 sec")
+        .build();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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 temporal 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();
+
+    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 DataflowRetrieval rawRetrieval;
+        if (specificationStrategy.equalsIgnoreCase(SPEC_FROM_FILE.getValue())) {
+            rawRetrieval = new FileSystemDataflowRetrieval();
+        } else {
+            rawRetrieval = new RegistryDataflowRetrieval(getLogger());
+        }
+
+        final DataflowRetrieval cachedRetrieval = new CachingDataflowRetrieval(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 {
+                final LogLevel logLevel = LogLevel.valueOf(bulletin.getLevel());
+                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 ignored) {
+                // There should be no malformed bulletin, but just in case the bulletin level, etc. ends up being null, catch the exception and ignore it,
+                // since there's not much we can do about it.
+            }
+        }
+    }
+
+    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, "flowfile-content");

Review comment:
       Is there any potential issue with using the same `flowfile-content` name for multiple processor invocations?  What do you think about including `ExecuteStateless` in the file name to help maintain tracking?

##########
File path: nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/ExecuteStateless.java
##########
@@ -0,0 +1,1202 @@
+/*
+ * 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 com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.module.jaxb.JaxbAnnotationIntrospector;
+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.ComponentLog;
+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.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.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 javax.net.ssl.SSLContext;
+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 {
+    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");
+    static final AllowableValue SPEC_FROM_REGISTRY = new AllowableValue("Use NiFi Registry", "Use NiFi Registry", "Dataflow to run is stored in NiFi Registry");
+
+    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.");
+    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.");
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    static final PropertyDescriptor DATAFLOW_TIMEOUT = new Builder()
+        .name("Dataflow Timeout")
+        .displayName("Dataflow Timeout")
+        .description("Specifies the maximum amount of time for an invocation of the stateless flow to complete. If the flow does not complete within this amount of time, the incoming FlowFile, if " +
+            "any, will be routed to the timeout relationship. In any case, the dataflow will be canceled and the invocation will end")
+        .required(true)
+        .addValidator(TIME_PERIOD_VALIDATOR)
+        .expressionLanguageSupported(FLOWFILE_ATTRIBUTES)
+        .defaultValue("60 sec")
+        .build();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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 temporal 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();
+
+    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 DataflowRetrieval rawRetrieval;
+        if (specificationStrategy.equalsIgnoreCase(SPEC_FROM_FILE.getValue())) {
+            rawRetrieval = new FileSystemDataflowRetrieval();
+        } else {
+            rawRetrieval = new RegistryDataflowRetrieval(getLogger());
+        }
+
+        final DataflowRetrieval cachedRetrieval = new CachingDataflowRetrieval(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 {
+                final LogLevel logLevel = LogLevel.valueOf(bulletin.getLevel());
+                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 ignored) {
+                // There should be no malformed bulletin, but just in case the bulletin level, etc. ends up being null, catch the exception and ignore it,
+                // since there's not much we can do about it.
+            }
+        }
+    }
+
+    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, "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 interface DataflowRetrieval {

Review comment:
       This internal interface and associated implementations keep everything together, but they also make it more difficult to unit test. What do you think about breaking out the interface and retrieval implementations to a separate package that could be unit-tested separately?

##########
File path: nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerLease.java
##########
@@ -210,21 +223,45 @@ boolean commit() {
             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.
              */
             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.

Review comment:
       Is this comment necessary in light of the log message?

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarClassLoaders.java
##########
@@ -472,10 +472,16 @@ private ClassLoader createBundleClassLoader(final BundleDetails bundleDetail, fi
      * @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 {

Review comment:
       As raised above, what do you think about renaming `log` to `logInfoEnabled` or something similar?

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarClassLoaders.java
##########
@@ -472,10 +472,16 @@ private ClassLoader createBundleClassLoader(final BundleDetails bundleDetail, fi
      * @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: " + narDirectory.getAbsolutePath() + " as class loader " + narClassLoader);

Review comment:
       This log method could be streamlined using placeholders:
   ```suggestion
               logger.info("Loaded NAR {} with {}", narDirectory.getAbsolutePath(), narClassLoader);
   ```

##########
File path: nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/ExecuteStateless.java
##########
@@ -0,0 +1,1202 @@
+/*
+ * 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 com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.module.jaxb.JaxbAnnotationIntrospector;
+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.ComponentLog;
+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.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.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 javax.net.ssl.SSLContext;
+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 {
+    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");
+    static final AllowableValue SPEC_FROM_REGISTRY = new AllowableValue("Use NiFi Registry", "Use NiFi Registry", "Dataflow to run is stored in NiFi Registry");
+
+    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.");
+    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.");
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    static final PropertyDescriptor DATAFLOW_TIMEOUT = new Builder()
+        .name("Dataflow Timeout")
+        .displayName("Dataflow Timeout")
+        .description("Specifies the maximum amount of time for an invocation of the stateless flow to complete. If the flow does not complete within this amount of time, the incoming FlowFile, if " +
+            "any, will be routed to the timeout relationship. In any case, the dataflow will be canceled and the invocation will end")
+        .required(true)
+        .addValidator(TIME_PERIOD_VALIDATOR)
+        .expressionLanguageSupported(FLOWFILE_ATTRIBUTES)
+        .defaultValue("60 sec")
+        .build();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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 temporal 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();
+
+    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 DataflowRetrieval rawRetrieval;
+        if (specificationStrategy.equalsIgnoreCase(SPEC_FROM_FILE.getValue())) {
+            rawRetrieval = new FileSystemDataflowRetrieval();
+        } else {
+            rawRetrieval = new RegistryDataflowRetrieval(getLogger());
+        }
+
+        final DataflowRetrieval cachedRetrieval = new CachingDataflowRetrieval(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 {
+                final LogLevel logLevel = LogLevel.valueOf(bulletin.getLevel());
+                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 ignored) {
+                // There should be no malformed bulletin, but just in case the bulletin level, etc. ends up being null, catch the exception and ignore it,
+                // since there's not much we can do about it.

Review comment:
       Is there any value in logging this exception condition as a debug or trace?

##########
File path: nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/ExecuteStateless.java
##########
@@ -0,0 +1,1202 @@
+/*
+ * 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 com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.module.jaxb.JaxbAnnotationIntrospector;
+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.ComponentLog;
+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.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.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 javax.net.ssl.SSLContext;
+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 {
+    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");
+    static final AllowableValue SPEC_FROM_REGISTRY = new AllowableValue("Use NiFi Registry", "Use NiFi Registry", "Dataflow to run is stored in NiFi Registry");
+
+    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.");
+    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.");
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    static final PropertyDescriptor DATAFLOW_TIMEOUT = new Builder()
+        .name("Dataflow Timeout")
+        .displayName("Dataflow Timeout")
+        .description("Specifies the maximum amount of time for an invocation of the stateless flow to complete. If the flow does not complete within this amount of time, the incoming FlowFile, if " +
+            "any, will be routed to the timeout relationship. In any case, the dataflow will be canceled and the invocation will end")
+        .required(true)
+        .addValidator(TIME_PERIOD_VALIDATOR)
+        .expressionLanguageSupported(FLOWFILE_ATTRIBUTES)
+        .defaultValue("60 sec")
+        .build();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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 temporal 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();
+
+    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)

Review comment:
       Although this seems like the safer approach, is there value in making this conditional on the property descriptor name, similar to how `DBCPConnectionPool` handles dynamic properties?

##########
File path: nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/ExecuteStateless.java
##########
@@ -0,0 +1,1202 @@
+/*
+ * 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 com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.module.jaxb.JaxbAnnotationIntrospector;
+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.ComponentLog;
+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.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.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 javax.net.ssl.SSLContext;
+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 {
+    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");
+    static final AllowableValue SPEC_FROM_REGISTRY = new AllowableValue("Use NiFi Registry", "Use NiFi Registry", "Dataflow to run is stored in NiFi Registry");
+
+    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.");
+    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.");
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    static final PropertyDescriptor DATAFLOW_TIMEOUT = new Builder()
+        .name("Dataflow Timeout")
+        .displayName("Dataflow Timeout")
+        .description("Specifies the maximum amount of time for an invocation of the stateless flow to complete. If the flow does not complete within this amount of time, the incoming FlowFile, if " +
+            "any, will be routed to the timeout relationship. In any case, the dataflow will be canceled and the invocation will end")
+        .required(true)
+        .addValidator(TIME_PERIOD_VALIDATOR)
+        .expressionLanguageSupported(FLOWFILE_ATTRIBUTES)
+        .defaultValue("60 sec")
+        .build();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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 temporal 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();
+
+    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 DataflowRetrieval rawRetrieval;
+        if (specificationStrategy.equalsIgnoreCase(SPEC_FROM_FILE.getValue())) {
+            rawRetrieval = new FileSystemDataflowRetrieval();
+        } else {
+            rawRetrieval = new RegistryDataflowRetrieval(getLogger());
+        }
+
+        final DataflowRetrieval cachedRetrieval = new CachingDataflowRetrieval(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);

Review comment:
       Since this is an info message, what do you think about including some additional details, such as the flow name, as part of the message?

##########
File path: nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/ExecuteStateless.java
##########
@@ -0,0 +1,1202 @@
+/*
+ * 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 com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.module.jaxb.JaxbAnnotationIntrospector;
+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.ComponentLog;
+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.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.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 javax.net.ssl.SSLContext;
+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 {
+    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");
+    static final AllowableValue SPEC_FROM_REGISTRY = new AllowableValue("Use NiFi Registry", "Use NiFi Registry", "Dataflow to run is stored in NiFi Registry");
+
+    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.");
+    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.");
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    static final PropertyDescriptor DATAFLOW_TIMEOUT = new Builder()
+        .name("Dataflow Timeout")
+        .displayName("Dataflow Timeout")
+        .description("Specifies the maximum amount of time for an invocation of the stateless flow to complete. If the flow does not complete within this amount of time, the incoming FlowFile, if " +
+            "any, will be routed to the timeout relationship. In any case, the dataflow will be canceled and the invocation will end")
+        .required(true)
+        .addValidator(TIME_PERIOD_VALIDATOR)
+        .expressionLanguageSupported(FLOWFILE_ATTRIBUTES)
+        .defaultValue("60 sec")
+        .build();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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 temporal 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();
+
+    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 DataflowRetrieval rawRetrieval;
+        if (specificationStrategy.equalsIgnoreCase(SPEC_FROM_FILE.getValue())) {
+            rawRetrieval = new FileSystemDataflowRetrieval();
+        } else {
+            rawRetrieval = new RegistryDataflowRetrieval(getLogger());
+        }
+
+        final DataflowRetrieval cachedRetrieval = new CachingDataflowRetrieval(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) {

Review comment:
       It looks like `createOutputFlowFiles` could throw other types of exceptions, should this catch block be more generic, or is the intention for other exceptions to bubble up and avoid the transfer to failure?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] gresockj commented on a change in pull request #5412: NIFI-9239: Updated Consume/Publish Kafka processors to support Exactl…

Posted by GitBox <gi...@apache.org>.
gresockj commented on a change in pull request #5412:
URL: https://github.com/apache/nifi/pull/5412#discussion_r716179469



##########
File path: 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,18 @@
             .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. Typically, we want this value set to true " +

Review comment:
       I'd rephrase to avoid "we" here.  Perhaps, "In the typical use case, this value is set to true..."
   
   I think we should also be more direct in tying this to Exactly Once in Stateless.  How about:
   
   "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."

##########
File path: nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaRecord_2_6.java
##########
@@ -536,6 +550,81 @@ public void process(final InputStream in) throws IOException {
         }
     }
 
+    private List<FlowFile> pollFlowFiles(final ProcessSession session) {
+        final List<FlowFile> initialFlowFiles = session.get(FlowFileFilters.newSizeBasedFilter(1, DataUnit.MB, 500));

Review comment:
       Is there any value in making this a processor property with a default of 500MB?  Otherwise, let's at least make this a constant.

##########
File path: nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafkaRecord_2_6.java
##########
@@ -536,6 +550,81 @@ public void process(final InputStream in) throws IOException {
         }
     }
 
+    private 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 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);

Review comment:
       Constant?

##########
File path: nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafka_2_6.java
##########
@@ -493,6 +517,82 @@ public void process(final InputStream rawIn) throws IOException {
         }
     }
 
+    private List<FlowFile> pollFlowFiles(final ProcessContext context, final ProcessSession session) {

Review comment:
       Is there any of this logic that can be reused in `PublishKafkaRecord_2_6"?  (this and the next few methods)

##########
File path: nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/ExecuteStateless.java
##########
@@ -0,0 +1,1202 @@
+/*
+ * 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 com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.module.jaxb.JaxbAnnotationIntrospector;
+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.ComponentLog;
+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.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.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 javax.net.ssl.SSLContext;
+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 {
+    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");
+    static final AllowableValue SPEC_FROM_REGISTRY = new AllowableValue("Use NiFi Registry", "Use NiFi Registry", "Dataflow to run is stored in NiFi Registry");
+
+    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.");
+    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.");
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    static final PropertyDescriptor DATAFLOW_TIMEOUT = new Builder()
+        .name("Dataflow Timeout")
+        .displayName("Dataflow Timeout")
+        .description("Specifies the maximum amount of time for an invocation of the stateless flow to complete. If the flow does not complete within this amount of time, the incoming FlowFile, if " +
+            "any, will be routed to the timeout relationship. In any case, the dataflow will be canceled and the invocation will end")
+        .required(true)
+        .addValidator(TIME_PERIOD_VALIDATOR)
+        .expressionLanguageSupported(FLOWFILE_ATTRIBUTES)
+        .defaultValue("60 sec")
+        .build();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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 temporal 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();
+
+    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.")

Review comment:
       Would it be worth defining a second failure relationship, perhaps "rejected" for this purpose?  This would help distinguish flowfiles that weren't processed from failures coming out of the data flow.  I suppose this case could be discerned from the lack of `failure.port.name` attribute, however.  Just wanted to pose the question.

##########
File path: 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,357 @@
+<!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 if using an external URL, if the Processor is stopped and started (or NiFi is restarted), it is possible that

Review comment:
       "if using an external URL or NiFi Registry"

##########
File path: nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/ExecuteStateless.java
##########
@@ -0,0 +1,1202 @@
+/*
+ * 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 com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.module.jaxb.JaxbAnnotationIntrospector;
+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.ComponentLog;
+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.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.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 javax.net.ssl.SSLContext;
+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 {
+    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");
+    static final AllowableValue SPEC_FROM_REGISTRY = new AllowableValue("Use NiFi Registry", "Use NiFi Registry", "Dataflow to run is stored in NiFi Registry");
+
+    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.");
+    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.");
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    static final PropertyDescriptor DATAFLOW_TIMEOUT = new Builder()
+        .name("Dataflow Timeout")
+        .displayName("Dataflow Timeout")
+        .description("Specifies the maximum amount of time for an invocation of the stateless flow to complete. If the flow does not complete within this amount of time, the incoming FlowFile, if " +
+            "any, will be routed to the timeout relationship. In any case, the dataflow will be canceled and the invocation will end")
+        .required(true)
+        .addValidator(TIME_PERIOD_VALIDATOR)
+        .expressionLanguageSupported(FLOWFILE_ATTRIBUTES)
+        .defaultValue("60 sec")
+        .build();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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 temporal 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();
+
+    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 DataflowRetrieval rawRetrieval;
+        if (specificationStrategy.equalsIgnoreCase(SPEC_FROM_FILE.getValue())) {
+            rawRetrieval = new FileSystemDataflowRetrieval();
+        } else {
+            rawRetrieval = new RegistryDataflowRetrieval(getLogger());
+        }
+
+        final DataflowRetrieval cachedRetrieval = new CachingDataflowRetrieval(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 {
+                final LogLevel logLevel = LogLevel.valueOf(bulletin.getLevel());
+                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 ignored) {
+                // There should be no malformed bulletin, but just in case the bulletin level, etc. ends up being null, catch the exception and ignore it,
+                // since there's not much we can do about it.
+            }
+        }
+    }
+
+    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, "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) {

Review comment:
       This is a nice touch.  Great idea to make the stateless flow searchable!  It looks nice in action.

##########
File path: nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/ExecuteStateless.java
##########
@@ -0,0 +1,1202 @@
+/*
+ * 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 com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.module.jaxb.JaxbAnnotationIntrospector;
+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.ComponentLog;
+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.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.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 javax.net.ssl.SSLContext;
+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 {
+    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");
+    static final AllowableValue SPEC_FROM_REGISTRY = new AllowableValue("Use NiFi Registry", "Use NiFi Registry", "Dataflow to run is stored in NiFi Registry");
+
+    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.");
+    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.");
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    static final PropertyDescriptor DATAFLOW_TIMEOUT = new Builder()
+        .name("Dataflow Timeout")
+        .displayName("Dataflow Timeout")
+        .description("Specifies the maximum amount of time for an invocation of the stateless flow to complete. If the flow does not complete within this amount of time, the incoming FlowFile, if " +
+            "any, will be routed to the timeout relationship. In any case, the dataflow will be canceled and the invocation will end")
+        .required(true)
+        .addValidator(TIME_PERIOD_VALIDATOR)
+        .expressionLanguageSupported(FLOWFILE_ATTRIBUTES)
+        .defaultValue("60 sec")
+        .build();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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 temporal 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();
+
+    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 DataflowRetrieval rawRetrieval;
+        if (specificationStrategy.equalsIgnoreCase(SPEC_FROM_FILE.getValue())) {
+            rawRetrieval = new FileSystemDataflowRetrieval();
+        } else {
+            rawRetrieval = new RegistryDataflowRetrieval(getLogger());
+        }
+
+        final DataflowRetrieval cachedRetrieval = new CachingDataflowRetrieval(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 {
+                final LogLevel logLevel = LogLevel.valueOf(bulletin.getLevel());
+                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 ignored) {
+                // There should be no malformed bulletin, but just in case the bulletin level, etc. ends up being null, catch the exception and ignore it,
+                // since there's not much we can do about it.
+            }
+        }
+    }
+
+    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, "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 interface DataflowRetrieval {
+        VersionedFlowSnapshot retrieveDataflowContents(ProcessContext context) throws IOException;
+    }
+
+    private static class FileSystemDataflowRetrieval implements DataflowRetrieval {
+        @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(DATAFLOW_FILE).asResource().read()) {
+                final VersionedFlowSnapshot versionedFlowSnapshot = objectMapper.readValue(in, VersionedFlowSnapshot.class);
+                return versionedFlowSnapshot;
+            }
+        }
+    }
+
+    private static class CachingDataflowRetrieval implements DataflowRetrieval {
+        private final String processorId;
+        private final ComponentLog logger;
+        private final DataflowRetrieval delegate;
+
+        public CachingDataflowRetrieval(final String processorId, final ComponentLog logger, final DataflowRetrieval delegate) {
+            this.processorId = processorId;
+            this.logger = logger;
+            this.delegate = delegate;
+        }
+
+        @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 from Registry. 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 from Registry but cannot create directory {} in order to cache the dataflow. " +
+                    "Upon restart, processor will not be able to function unless Registry is available", cacheFile);
+                return;
+            }
+
+            try {
+                final ObjectMapper objectMapper = new ObjectMapper();

Review comment:
       Could the `objectMapper` be moved to a private field so you don't have to configure it in two places?

##########
File path: nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/ExecuteStateless.java
##########
@@ -0,0 +1,1202 @@
+/*
+ * 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 com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.module.jaxb.JaxbAnnotationIntrospector;
+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.ComponentLog;
+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.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.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 javax.net.ssl.SSLContext;
+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 {
+    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");
+    static final AllowableValue SPEC_FROM_REGISTRY = new AllowableValue("Use NiFi Registry", "Use NiFi Registry", "Dataflow to run is stored in NiFi Registry");
+
+    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.");
+    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.");
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    static final PropertyDescriptor DATAFLOW_TIMEOUT = new Builder()
+        .name("Dataflow Timeout")
+        .displayName("Dataflow Timeout")
+        .description("Specifies the maximum amount of time for an invocation of the stateless flow to complete. If the flow does not complete within this amount of time, the incoming FlowFile, if " +
+            "any, will be routed to the timeout relationship. In any case, the dataflow will be canceled and the invocation will end")
+        .required(true)
+        .addValidator(TIME_PERIOD_VALIDATOR)
+        .expressionLanguageSupported(FLOWFILE_ATTRIBUTES)
+        .defaultValue("60 sec")
+        .build();
+
+    static final PropertyDescriptor LIB_DIRECTORY = new Builder()
+        .name("NAR Directory")
+        .displayName("NAR Directory")
+        .description("The directory to retrieve NAR's from")

Review comment:
       No apostrophe here

##########
File path: nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/ExecuteStateless.java
##########
@@ -0,0 +1,1202 @@
+/*
+ * 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 com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.module.jaxb.JaxbAnnotationIntrospector;
+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.ComponentLog;
+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.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.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 javax.net.ssl.SSLContext;
+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 {
+    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");
+    static final AllowableValue SPEC_FROM_REGISTRY = new AllowableValue("Use NiFi Registry", "Use NiFi Registry", "Dataflow to run is stored in NiFi Registry");
+
+    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.");
+    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.");
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    static final PropertyDescriptor DATAFLOW_TIMEOUT = new Builder()
+        .name("Dataflow Timeout")
+        .displayName("Dataflow Timeout")
+        .description("Specifies the maximum amount of time for an invocation of the stateless flow to complete. If the flow does not complete within this amount of time, the incoming FlowFile, if " +
+            "any, will be routed to the timeout relationship. In any case, the dataflow will be canceled and the invocation will end")

Review comment:
       Missing a period.  Also, what do you mean by "in any case"?  It seems clearer to say: "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."

##########
File path: nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/ExecuteStateless.java
##########
@@ -0,0 +1,1202 @@
+/*
+ * 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 com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.module.jaxb.JaxbAnnotationIntrospector;
+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.ComponentLog;
+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.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.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 javax.net.ssl.SSLContext;
+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 {
+    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");
+    static final AllowableValue SPEC_FROM_REGISTRY = new AllowableValue("Use NiFi Registry", "Use NiFi Registry", "Dataflow to run is stored in NiFi Registry");
+
+    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.");
+    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.");
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    static final PropertyDescriptor DATAFLOW_TIMEOUT = new Builder()
+        .name("Dataflow Timeout")
+        .displayName("Dataflow Timeout")
+        .description("Specifies the maximum amount of time for an invocation of the stateless flow to complete. If the flow does not complete within this amount of time, the incoming FlowFile, if " +
+            "any, will be routed to the timeout relationship. In any case, the dataflow will be canceled and the invocation will end")
+        .required(true)
+        .addValidator(TIME_PERIOD_VALIDATOR)
+        .expressionLanguageSupported(FLOWFILE_ATTRIBUTES)
+        .defaultValue("60 sec")
+        .build();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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 temporal 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();
+
+    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 DataflowRetrieval rawRetrieval;
+        if (specificationStrategy.equalsIgnoreCase(SPEC_FROM_FILE.getValue())) {
+            rawRetrieval = new FileSystemDataflowRetrieval();
+        } else {
+            rawRetrieval = new RegistryDataflowRetrieval(getLogger());
+        }
+
+        final DataflowRetrieval cachedRetrieval = new CachingDataflowRetrieval(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();

Review comment:
       I may be missing something, but I couldn't get any bulletins to appear on the ExecuteStateless processor when running a dataflow that produced an ERROR bulletin -- `maxBulletinId` was always -1 here during my testing.

##########
File path: nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/ExecuteStateless.java
##########
@@ -0,0 +1,1202 @@
+/*
+ * 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 com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.module.jaxb.JaxbAnnotationIntrospector;
+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.ComponentLog;
+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.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.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 javax.net.ssl.SSLContext;
+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 {
+    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");
+    static final AllowableValue SPEC_FROM_REGISTRY = new AllowableValue("Use NiFi Registry", "Use NiFi Registry", "Dataflow to run is stored in NiFi Registry");
+
+    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.");
+    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.");
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    static final PropertyDescriptor DATAFLOW_TIMEOUT = new Builder()
+        .name("Dataflow Timeout")
+        .displayName("Dataflow Timeout")
+        .description("Specifies the maximum amount of time for an invocation of the stateless flow to complete. If the flow does not complete within this amount of time, the incoming FlowFile, if " +
+            "any, will be routed to the timeout relationship. In any case, the dataflow will be canceled and the invocation will end")
+        .required(true)
+        .addValidator(TIME_PERIOD_VALIDATOR)
+        .expressionLanguageSupported(FLOWFILE_ATTRIBUTES)
+        .defaultValue("60 sec")
+        .build();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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 temporal and may be deleted at " +

Review comment:
       temporary instead of temporal

##########
File path: 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,357 @@
+<!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 if using an external URL, 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).
+</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 make

Review comment:
       Remove "make" from "can make provide"

##########
File path: nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/ExecuteStateless.java
##########
@@ -0,0 +1,1202 @@
+/*
+ * 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 com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.module.jaxb.JaxbAnnotationIntrospector;
+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.ComponentLog;
+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.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.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 javax.net.ssl.SSLContext;
+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 {
+    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");
+    static final AllowableValue SPEC_FROM_REGISTRY = new AllowableValue("Use NiFi Registry", "Use NiFi Registry", "Dataflow to run is stored in NiFi Registry");
+
+    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.");
+    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.");
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    static final PropertyDescriptor DATAFLOW_TIMEOUT = new Builder()
+        .name("Dataflow Timeout")
+        .displayName("Dataflow Timeout")
+        .description("Specifies the maximum amount of time for an invocation of the stateless flow to complete. If the flow does not complete within this amount of time, the incoming FlowFile, if " +
+            "any, will be routed to the timeout relationship. In any case, the dataflow will be canceled and the invocation will end")
+        .required(true)
+        .addValidator(TIME_PERIOD_VALIDATOR)
+        .expressionLanguageSupported(FLOWFILE_ATTRIBUTES)
+        .defaultValue("60 sec")
+        .build();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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 temporal 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();
+
+    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 DataflowRetrieval rawRetrieval;
+        if (specificationStrategy.equalsIgnoreCase(SPEC_FROM_FILE.getValue())) {
+            rawRetrieval = new FileSystemDataflowRetrieval();
+        } else {
+            rawRetrieval = new RegistryDataflowRetrieval(getLogger());
+        }
+
+        final DataflowRetrieval cachedRetrieval = new CachingDataflowRetrieval(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 {
+                final LogLevel logLevel = LogLevel.valueOf(bulletin.getLevel());
+                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 ignored) {
+                // There should be no malformed bulletin, but just in case the bulletin level, etc. ends up being null, catch the exception and ignore it,
+                // since there's not much we can do about it.
+            }
+        }
+    }
+
+    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, "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 interface DataflowRetrieval {
+        VersionedFlowSnapshot retrieveDataflowContents(ProcessContext context) throws IOException;
+    }
+
+    private static class FileSystemDataflowRetrieval implements DataflowRetrieval {
+        @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(DATAFLOW_FILE).asResource().read()) {
+                final VersionedFlowSnapshot versionedFlowSnapshot = objectMapper.readValue(in, VersionedFlowSnapshot.class);
+                return versionedFlowSnapshot;
+            }
+        }
+    }
+
+    private static class CachingDataflowRetrieval implements DataflowRetrieval {
+        private final String processorId;
+        private final ComponentLog logger;
+        private final DataflowRetrieval delegate;
+
+        public CachingDataflowRetrieval(final String processorId, final ComponentLog logger, final DataflowRetrieval delegate) {
+            this.processorId = processorId;
+            this.logger = logger;
+            this.delegate = delegate;
+        }
+
+        @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 from Registry. Will restore Flow Snapshot from cached version at {}", cacheFile.getAbsolutePath(), e);

Review comment:
       "Flow"




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] gresockj commented on a change in pull request #5412: NIFI-9239: Updated Consume/Publish Kafka processors to support Exactl…

Posted by GitBox <gi...@apache.org>.
gresockj commented on a change in pull request #5412:
URL: https://github.com/apache/nifi/pull/5412#discussion_r716980519



##########
File path: nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/ExecuteStateless.java
##########
@@ -0,0 +1,1202 @@
+/*
+ * 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 com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.module.jaxb.JaxbAnnotationIntrospector;
+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.ComponentLog;
+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.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.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 javax.net.ssl.SSLContext;
+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 {
+    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");
+    static final AllowableValue SPEC_FROM_REGISTRY = new AllowableValue("Use NiFi Registry", "Use NiFi Registry", "Dataflow to run is stored in NiFi Registry");
+
+    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.");
+    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.");
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    static final PropertyDescriptor DATAFLOW_TIMEOUT = new Builder()
+        .name("Dataflow Timeout")
+        .displayName("Dataflow Timeout")
+        .description("Specifies the maximum amount of time for an invocation of the stateless flow to complete. If the flow does not complete within this amount of time, the incoming FlowFile, if " +
+            "any, will be routed to the timeout relationship. In any case, the dataflow will be canceled and the invocation will end")
+        .required(true)
+        .addValidator(TIME_PERIOD_VALIDATOR)
+        .expressionLanguageSupported(FLOWFILE_ATTRIBUTES)
+        .defaultValue("60 sec")
+        .build();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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 temporal 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();
+
+    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 DataflowRetrieval rawRetrieval;
+        if (specificationStrategy.equalsIgnoreCase(SPEC_FROM_FILE.getValue())) {
+            rawRetrieval = new FileSystemDataflowRetrieval();
+        } else {
+            rawRetrieval = new RegistryDataflowRetrieval(getLogger());
+        }
+
+        final DataflowRetrieval cachedRetrieval = new CachingDataflowRetrieval(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 {
+                final LogLevel logLevel = LogLevel.valueOf(bulletin.getLevel());
+                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 ignored) {
+                // There should be no malformed bulletin, but just in case the bulletin level, etc. ends up being null, catch the exception and ignore it,
+                // since there's not much we can do about it.
+            }
+        }
+    }
+
+    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, "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 interface DataflowRetrieval {

Review comment:
       I actually had the same thought when initially reading through the code.  Might such a controller service be useful outside of ExecuteStateless?  Not sure if this would be overkill, but worth considering.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] exceptionfactory commented on a change in pull request #5412: NIFI-9239: Updated Consume/Publish Kafka processors to support Exactl…

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on a change in pull request #5412:
URL: https://github.com/apache/nifi/pull/5412#discussion_r716947906



##########
File path: nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/ExecuteStateless.java
##########
@@ -0,0 +1,1202 @@
+/*
+ * 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 com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.module.jaxb.JaxbAnnotationIntrospector;
+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.ComponentLog;
+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.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.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 javax.net.ssl.SSLContext;
+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 {
+    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");
+    static final AllowableValue SPEC_FROM_REGISTRY = new AllowableValue("Use NiFi Registry", "Use NiFi Registry", "Dataflow to run is stored in NiFi Registry");
+
+    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.");
+    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.");
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    static final PropertyDescriptor DATAFLOW_TIMEOUT = new Builder()
+        .name("Dataflow Timeout")
+        .displayName("Dataflow Timeout")
+        .description("Specifies the maximum amount of time for an invocation of the stateless flow to complete. If the flow does not complete within this amount of time, the incoming FlowFile, if " +
+            "any, will be routed to the timeout relationship. In any case, the dataflow will be canceled and the invocation will end")
+        .required(true)
+        .addValidator(TIME_PERIOD_VALIDATOR)
+        .expressionLanguageSupported(FLOWFILE_ATTRIBUTES)
+        .defaultValue("60 sec")
+        .build();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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 temporal 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();
+
+    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 DataflowRetrieval rawRetrieval;
+        if (specificationStrategy.equalsIgnoreCase(SPEC_FROM_FILE.getValue())) {
+            rawRetrieval = new FileSystemDataflowRetrieval();
+        } else {
+            rawRetrieval = new RegistryDataflowRetrieval(getLogger());
+        }
+
+        final DataflowRetrieval cachedRetrieval = new CachingDataflowRetrieval(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) {

Review comment:
       Thanks for the confirmation, just wanted to confirm that any potential RuntimeExceptions should bubble up as opposed to routing to failure.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] exceptionfactory commented on a change in pull request #5412: NIFI-9239: Updated Consume/Publish Kafka processors to support Exactl…

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on a change in pull request #5412:
URL: https://github.com/apache/nifi/pull/5412#discussion_r716944946



##########
File path: nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/ExecuteStateless.java
##########
@@ -0,0 +1,1202 @@
+/*
+ * 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 com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.module.jaxb.JaxbAnnotationIntrospector;
+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.ComponentLog;
+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.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.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 javax.net.ssl.SSLContext;
+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 {
+    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");
+    static final AllowableValue SPEC_FROM_REGISTRY = new AllowableValue("Use NiFi Registry", "Use NiFi Registry", "Dataflow to run is stored in NiFi Registry");
+
+    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.");
+    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.");
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    static final PropertyDescriptor DATAFLOW_TIMEOUT = new Builder()
+        .name("Dataflow Timeout")
+        .displayName("Dataflow Timeout")
+        .description("Specifies the maximum amount of time for an invocation of the stateless flow to complete. If the flow does not complete within this amount of time, the incoming FlowFile, if " +
+            "any, will be routed to the timeout relationship. In any case, the dataflow will be canceled and the invocation will end")
+        .required(true)
+        .addValidator(TIME_PERIOD_VALIDATOR)
+        .expressionLanguageSupported(FLOWFILE_ATTRIBUTES)
+        .defaultValue("60 sec")
+        .build();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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 temporal 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();
+
+    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 DataflowRetrieval rawRetrieval;
+        if (specificationStrategy.equalsIgnoreCase(SPEC_FROM_FILE.getValue())) {
+            rawRetrieval = new FileSystemDataflowRetrieval();
+        } else {
+            rawRetrieval = new RegistryDataflowRetrieval(getLogger());
+        }
+
+        final DataflowRetrieval cachedRetrieval = new CachingDataflowRetrieval(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 {
+                final LogLevel logLevel = LogLevel.valueOf(bulletin.getLevel());
+                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 ignored) {
+                // There should be no malformed bulletin, but just in case the bulletin level, etc. ends up being null, catch the exception and ignore it,
+                // since there's not much we can do about it.
+            }
+        }
+    }
+
+    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, "flowfile-content");

Review comment:
       Thanks for the clarification, that makes sense!

##########
File path: nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/ExecuteStateless.java
##########
@@ -0,0 +1,1202 @@
+/*
+ * 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 com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.module.jaxb.JaxbAnnotationIntrospector;
+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.ComponentLog;
+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.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.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 javax.net.ssl.SSLContext;
+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 {
+    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");
+    static final AllowableValue SPEC_FROM_REGISTRY = new AllowableValue("Use NiFi Registry", "Use NiFi Registry", "Dataflow to run is stored in NiFi Registry");
+
+    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.");
+    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.");
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    static final PropertyDescriptor DATAFLOW_TIMEOUT = new Builder()
+        .name("Dataflow Timeout")
+        .displayName("Dataflow Timeout")
+        .description("Specifies the maximum amount of time for an invocation of the stateless flow to complete. If the flow does not complete within this amount of time, the incoming FlowFile, if " +
+            "any, will be routed to the timeout relationship. In any case, the dataflow will be canceled and the invocation will end")
+        .required(true)
+        .addValidator(TIME_PERIOD_VALIDATOR)
+        .expressionLanguageSupported(FLOWFILE_ATTRIBUTES)
+        .defaultValue("60 sec")
+        .build();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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 temporal 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();
+
+    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 DataflowRetrieval rawRetrieval;
+        if (specificationStrategy.equalsIgnoreCase(SPEC_FROM_FILE.getValue())) {
+            rawRetrieval = new FileSystemDataflowRetrieval();
+        } else {
+            rawRetrieval = new RegistryDataflowRetrieval(getLogger());
+        }
+
+        final DataflowRetrieval cachedRetrieval = new CachingDataflowRetrieval(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) {

Review comment:
       Thanks for the confirmation, just wanted to confirm that any potential RuntimeExceptions should bubble up as opposed to routing to failure.

##########
File path: nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/ExecuteStateless.java
##########
@@ -0,0 +1,1202 @@
+/*
+ * 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 com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.module.jaxb.JaxbAnnotationIntrospector;
+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.ComponentLog;
+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.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.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 javax.net.ssl.SSLContext;
+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 {
+    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");
+    static final AllowableValue SPEC_FROM_REGISTRY = new AllowableValue("Use NiFi Registry", "Use NiFi Registry", "Dataflow to run is stored in NiFi Registry");
+
+    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.");
+    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.");
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    static final PropertyDescriptor DATAFLOW_TIMEOUT = new Builder()
+        .name("Dataflow Timeout")
+        .displayName("Dataflow Timeout")
+        .description("Specifies the maximum amount of time for an invocation of the stateless flow to complete. If the flow does not complete within this amount of time, the incoming FlowFile, if " +
+            "any, will be routed to the timeout relationship. In any case, the dataflow will be canceled and the invocation will end")
+        .required(true)
+        .addValidator(TIME_PERIOD_VALIDATOR)
+        .expressionLanguageSupported(FLOWFILE_ATTRIBUTES)
+        .defaultValue("60 sec")
+        .build();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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 temporal 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();
+
+    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 DataflowRetrieval rawRetrieval;
+        if (specificationStrategy.equalsIgnoreCase(SPEC_FROM_FILE.getValue())) {
+            rawRetrieval = new FileSystemDataflowRetrieval();
+        } else {
+            rawRetrieval = new RegistryDataflowRetrieval(getLogger());
+        }
+
+        final DataflowRetrieval cachedRetrieval = new CachingDataflowRetrieval(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 {
+                final LogLevel logLevel = LogLevel.valueOf(bulletin.getLevel());
+                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 ignored) {
+                // There should be no malformed bulletin, but just in case the bulletin level, etc. ends up being null, catch the exception and ignore it,
+                // since there's not much we can do about it.
+            }
+        }
+    }
+
+    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, "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 interface DataflowRetrieval {

Review comment:
       On further consideration, what do you think about turning this into a Controller Service interface?  File-based and Registry-based implementations should cover most use cases, but that would allow for custom implementations using other approaches.  That would also remove the direct dependency on nifi-registry, and also remove the Registry SSL Context Service property.

##########
File path: nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/ExecuteStateless.java
##########
@@ -0,0 +1,1202 @@
+/*
+ * 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 com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.module.jaxb.JaxbAnnotationIntrospector;
+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.ComponentLog;
+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.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.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 javax.net.ssl.SSLContext;
+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 {
+    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");
+    static final AllowableValue SPEC_FROM_REGISTRY = new AllowableValue("Use NiFi Registry", "Use NiFi Registry", "Dataflow to run is stored in NiFi Registry");
+
+    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.");
+    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.");
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    static final PropertyDescriptor DATAFLOW_TIMEOUT = new Builder()
+        .name("Dataflow Timeout")
+        .displayName("Dataflow Timeout")
+        .description("Specifies the maximum amount of time for an invocation of the stateless flow to complete. If the flow does not complete within this amount of time, the incoming FlowFile, if " +
+            "any, will be routed to the timeout relationship. In any case, the dataflow will be canceled and the invocation will end")
+        .required(true)
+        .addValidator(TIME_PERIOD_VALIDATOR)
+        .expressionLanguageSupported(FLOWFILE_ATTRIBUTES)
+        .defaultValue("60 sec")
+        .build();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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 temporal 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();
+
+    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 DataflowRetrieval rawRetrieval;
+        if (specificationStrategy.equalsIgnoreCase(SPEC_FROM_FILE.getValue())) {
+            rawRetrieval = new FileSystemDataflowRetrieval();
+        } else {
+            rawRetrieval = new RegistryDataflowRetrieval(getLogger());
+        }
+
+        final DataflowRetrieval cachedRetrieval = new CachingDataflowRetrieval(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 {
+                final LogLevel logLevel = LogLevel.valueOf(bulletin.getLevel());
+                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 ignored) {
+                // There should be no malformed bulletin, but just in case the bulletin level, etc. ends up being null, catch the exception and ignore it,
+                // since there's not much we can do about it.
+            }
+        }
+    }
+
+    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, "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 interface DataflowRetrieval {

Review comment:
       Could the the `Dataflow File/URL` property be repurposed to parameterize the Bucket, Flow, and Version properties? I understand the tradeoff of complexity, but the current implementation requires a number of properties that are specific to Registry.  The `dependsOn` element helps keep this more manageable, but it does add a good bit of code complexity.  It seems like another potential implementation could be direct retrieval from a Git repository or web service. If those additional implementations seem unlikely, or would not introduce too much complication to the current approach, then a Controller Service seems unnecessary.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] markap14 commented on a change in pull request #5412: NIFI-9239: Updated Consume/Publish Kafka processors to support Exactl…

Posted by GitBox <gi...@apache.org>.
markap14 commented on a change in pull request #5412:
URL: https://github.com/apache/nifi/pull/5412#discussion_r716939781



##########
File path: nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/ExecuteStateless.java
##########
@@ -0,0 +1,1202 @@
+/*
+ * 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 com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.module.jaxb.JaxbAnnotationIntrospector;
+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.ComponentLog;
+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.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.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 javax.net.ssl.SSLContext;
+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 {
+    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");
+    static final AllowableValue SPEC_FROM_REGISTRY = new AllowableValue("Use NiFi Registry", "Use NiFi Registry", "Dataflow to run is stored in NiFi Registry");
+
+    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.");
+    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.");
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    static final PropertyDescriptor DATAFLOW_TIMEOUT = new Builder()
+        .name("Dataflow Timeout")
+        .displayName("Dataflow Timeout")
+        .description("Specifies the maximum amount of time for an invocation of the stateless flow to complete. If the flow does not complete within this amount of time, the incoming FlowFile, if " +
+            "any, will be routed to the timeout relationship. In any case, the dataflow will be canceled and the invocation will end")
+        .required(true)
+        .addValidator(TIME_PERIOD_VALIDATOR)
+        .expressionLanguageSupported(FLOWFILE_ATTRIBUTES)
+        .defaultValue("60 sec")
+        .build();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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 temporal 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();
+
+    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 DataflowRetrieval rawRetrieval;
+        if (specificationStrategy.equalsIgnoreCase(SPEC_FROM_FILE.getValue())) {
+            rawRetrieval = new FileSystemDataflowRetrieval();
+        } else {
+            rawRetrieval = new RegistryDataflowRetrieval(getLogger());
+        }
+
+        final DataflowRetrieval cachedRetrieval = new CachingDataflowRetrieval(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 {
+                final LogLevel logLevel = LogLevel.valueOf(bulletin.getLevel());
+                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 ignored) {
+                // There should be no malformed bulletin, but just in case the bulletin level, etc. ends up being null, catch the exception and ignore it,
+                // since there's not much we can do about it.

Review comment:
       Yeah, this should never happen. If it does, i'll actually log a warning.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] markap14 commented on a change in pull request #5412: NIFI-9239: Updated Consume/Publish Kafka processors to support Exactl…

Posted by GitBox <gi...@apache.org>.
markap14 commented on a change in pull request #5412:
URL: https://github.com/apache/nifi/pull/5412#discussion_r716945378



##########
File path: nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/ExecuteStateless.java
##########
@@ -0,0 +1,1202 @@
+/*
+ * 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 com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.module.jaxb.JaxbAnnotationIntrospector;
+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.ComponentLog;
+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.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.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 javax.net.ssl.SSLContext;
+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 {
+    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");
+    static final AllowableValue SPEC_FROM_REGISTRY = new AllowableValue("Use NiFi Registry", "Use NiFi Registry", "Dataflow to run is stored in NiFi Registry");
+
+    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.");
+    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.");
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    static final PropertyDescriptor DATAFLOW_TIMEOUT = new Builder()
+        .name("Dataflow Timeout")
+        .displayName("Dataflow Timeout")
+        .description("Specifies the maximum amount of time for an invocation of the stateless flow to complete. If the flow does not complete within this amount of time, the incoming FlowFile, if " +
+            "any, will be routed to the timeout relationship. In any case, the dataflow will be canceled and the invocation will end")
+        .required(true)
+        .addValidator(TIME_PERIOD_VALIDATOR)
+        .expressionLanguageSupported(FLOWFILE_ATTRIBUTES)
+        .defaultValue("60 sec")
+        .build();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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 temporal 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();
+
+    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 DataflowRetrieval rawRetrieval;
+        if (specificationStrategy.equalsIgnoreCase(SPEC_FROM_FILE.getValue())) {
+            rawRetrieval = new FileSystemDataflowRetrieval();
+        } else {
+            rawRetrieval = new RegistryDataflowRetrieval(getLogger());
+        }
+
+        final DataflowRetrieval cachedRetrieval = new CachingDataflowRetrieval(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 {
+                final LogLevel logLevel = LogLevel.valueOf(bulletin.getLevel());
+                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 ignored) {
+                // There should be no malformed bulletin, but just in case the bulletin level, etc. ends up being null, catch the exception and ignore it,
+                // since there's not much we can do about it.
+            }
+        }
+    }
+
+    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, "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 interface DataflowRetrieval {

Review comment:
       Fair enough.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] markap14 commented on pull request #5412: NIFI-9239: Updated Consume/Publish Kafka processors to support Exactl…

Posted by GitBox <gi...@apache.org>.
markap14 commented on pull request #5412:
URL: https://github.com/apache/nifi/pull/5412#issuecomment-929358954


   @gresockj @exceptionfactory  huge thanks for all of the very thoughtful and thorough feedback! I will push a new commit momentarily that should address all concerns. And I definitely agree @exceptionfactory that there are a lot more things that should be unit tested. I intended to write a bunch more tests and then it totally slipped my mind. So the next commit will include new tests, mostly around error conditions, such as timeouts, processors that throw exceptions, failure ports, etc. Thanks!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] markap14 commented on a change in pull request #5412: NIFI-9239: Updated Consume/Publish Kafka processors to support Exactl…

Posted by GitBox <gi...@apache.org>.
markap14 commented on a change in pull request #5412:
URL: https://github.com/apache/nifi/pull/5412#discussion_r719597117



##########
File path: nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/retrieval/CachingDataflowRetrieval.java
##########
@@ -0,0 +1,92 @@
+/*
+ * 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;
+
+public class CachingDataflowRetrieval implements DataflowRetrieval {
+    private final String processorId;
+    private final ComponentLog logger;
+    private final DataflowRetrieval delegate;
+    private final ObjectMapper objectMapper;
+
+
+    public CachingDataflowRetrieval(final String processorId, final ComponentLog logger, final DataflowRetrieval 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 {

Review comment:
       This isn't a concern. It's fetched only in the @OnScheduled, which is single-threaded




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] markap14 commented on a change in pull request #5412: NIFI-9239: Updated Consume/Publish Kafka processors to support Exactl…

Posted by GitBox <gi...@apache.org>.
markap14 commented on a change in pull request #5412:
URL: https://github.com/apache/nifi/pull/5412#discussion_r719600576



##########
File path: nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/retrieval/DataflowRetrieval.java
##########
@@ -0,0 +1,27 @@
+/*
+ * 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.processor.ProcessContext;
+import org.apache.nifi.registry.flow.VersionedFlowSnapshot;
+
+import java.io.IOException;
+
+public interface DataflowRetrieval {

Review comment:
       Yeah I'm ok with `DataflowProvider`




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] markap14 commented on a change in pull request #5412: NIFI-9239: Updated Consume/Publish Kafka processors to support Exactl…

Posted by GitBox <gi...@apache.org>.
markap14 commented on a change in pull request #5412:
URL: https://github.com/apache/nifi/pull/5412#discussion_r719601610



##########
File path: nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/retrieval/FileSystemDataflowRetrieval.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 FileSystemDataflowRetrieval implements DataflowRetrieval {
+    @Override
+    public VersionedFlowSnapshot retrieveDataflowContents(final ProcessContext context) throws IOException {
+        final ObjectMapper objectMapper = new ObjectMapper();
+        objectMapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);

Review comment:
       It's not needed in this case. It's needed for the caching case because when a dataflow is retrieved from registry, there is more information available in the dataflow snapshot wrapper element, which is not provided when we download a flow locally.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] markap14 commented on a change in pull request #5412: NIFI-9239: Updated Consume/Publish Kafka processors to support Exactl…

Posted by GitBox <gi...@apache.org>.
markap14 commented on a change in pull request #5412:
URL: https://github.com/apache/nifi/pull/5412#discussion_r716927743



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarClassLoaders.java
##########
@@ -140,8 +140,8 @@ public void init(final ClassLoader rootClassloader,
     /**
      * 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)

Review comment:
       I considered changing it to something like that but felt like it was just more confusing. If `true`, it logs at info level. That's all that'll generally show up in logs. So for any typical deployment, `false` = do not log. I.e., "Do you want the details showing up in typical logs or no?" 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] markap14 commented on a change in pull request #5412: NIFI-9239: Updated Consume/Publish Kafka processors to support Exactl…

Posted by GitBox <gi...@apache.org>.
markap14 commented on a change in pull request #5412:
URL: https://github.com/apache/nifi/pull/5412#discussion_r716927743



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarClassLoaders.java
##########
@@ -140,8 +140,8 @@ public void init(final ClassLoader rootClassloader,
     /**
      * 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)

Review comment:
       I considered changing it to something like that but felt like it was just more confusing. If `true`, it logs at info level. That's all that'll generally show up in logs. So for any typical deployment, `false` = do not log. I.e., "Do you want the details showing up in typical logs or no?" 

##########
File path: nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/ExecuteStateless.java
##########
@@ -0,0 +1,1202 @@
+/*
+ * 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 com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.module.jaxb.JaxbAnnotationIntrospector;
+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.ComponentLog;
+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.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.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 javax.net.ssl.SSLContext;
+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 {
+    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");
+    static final AllowableValue SPEC_FROM_REGISTRY = new AllowableValue("Use NiFi Registry", "Use NiFi Registry", "Dataflow to run is stored in NiFi Registry");
+
+    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.");
+    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.");
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    static final PropertyDescriptor DATAFLOW_TIMEOUT = new Builder()
+        .name("Dataflow Timeout")
+        .displayName("Dataflow Timeout")
+        .description("Specifies the maximum amount of time for an invocation of the stateless flow to complete. If the flow does not complete within this amount of time, the incoming FlowFile, if " +
+            "any, will be routed to the timeout relationship. In any case, the dataflow will be canceled and the invocation will end")
+        .required(true)
+        .addValidator(TIME_PERIOD_VALIDATOR)
+        .expressionLanguageSupported(FLOWFILE_ATTRIBUTES)
+        .defaultValue("60 sec")
+        .build();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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 temporal 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();
+
+    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)

Review comment:
       I did consider that. But I found that to be very confusing and not very straight forward. It also adds complexity in that the person who is to run this component needs to know whether or not each parameter is sensitive rather just knowing which parameters need to be set. I could probably be convinced to go the route of checking if the property starts with "Sensitive" but I think I lean a bit more toward this approach.

##########
File path: nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/ExecuteStateless.java
##########
@@ -0,0 +1,1202 @@
+/*
+ * 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 com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.module.jaxb.JaxbAnnotationIntrospector;
+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.ComponentLog;
+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.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.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 javax.net.ssl.SSLContext;
+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 {
+    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");
+    static final AllowableValue SPEC_FROM_REGISTRY = new AllowableValue("Use NiFi Registry", "Use NiFi Registry", "Dataflow to run is stored in NiFi Registry");
+
+    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.");
+    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.");
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    static final PropertyDescriptor DATAFLOW_TIMEOUT = new Builder()
+        .name("Dataflow Timeout")
+        .displayName("Dataflow Timeout")
+        .description("Specifies the maximum amount of time for an invocation of the stateless flow to complete. If the flow does not complete within this amount of time, the incoming FlowFile, if " +
+            "any, will be routed to the timeout relationship. In any case, the dataflow will be canceled and the invocation will end")
+        .required(true)
+        .addValidator(TIME_PERIOD_VALIDATOR)
+        .expressionLanguageSupported(FLOWFILE_ATTRIBUTES)
+        .defaultValue("60 sec")
+        .build();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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 temporal 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();
+
+    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 DataflowRetrieval rawRetrieval;
+        if (specificationStrategy.equalsIgnoreCase(SPEC_FROM_FILE.getValue())) {
+            rawRetrieval = new FileSystemDataflowRetrieval();
+        } else {
+            rawRetrieval = new RegistryDataflowRetrieval(getLogger());
+        }
+
+        final DataflowRetrieval cachedRetrieval = new CachingDataflowRetrieval(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);

Review comment:
       I like the idea. But when the flow is saved via NiFi's  Download Flow Definition, what we get is basically an empty VersionedFlowSnapshot wrapper, with only the "flowContents" populated. So the name of the flow, etc., are not available.

##########
File path: nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/ExecuteStateless.java
##########
@@ -0,0 +1,1202 @@
+/*
+ * 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 com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.module.jaxb.JaxbAnnotationIntrospector;
+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.ComponentLog;
+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.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.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 javax.net.ssl.SSLContext;
+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 {
+    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");
+    static final AllowableValue SPEC_FROM_REGISTRY = new AllowableValue("Use NiFi Registry", "Use NiFi Registry", "Dataflow to run is stored in NiFi Registry");
+
+    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.");
+    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.");
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    static final PropertyDescriptor DATAFLOW_TIMEOUT = new Builder()
+        .name("Dataflow Timeout")
+        .displayName("Dataflow Timeout")
+        .description("Specifies the maximum amount of time for an invocation of the stateless flow to complete. If the flow does not complete within this amount of time, the incoming FlowFile, if " +
+            "any, will be routed to the timeout relationship. In any case, the dataflow will be canceled and the invocation will end")
+        .required(true)
+        .addValidator(TIME_PERIOD_VALIDATOR)
+        .expressionLanguageSupported(FLOWFILE_ATTRIBUTES)
+        .defaultValue("60 sec")
+        .build();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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 temporal 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();
+
+    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 DataflowRetrieval rawRetrieval;
+        if (specificationStrategy.equalsIgnoreCase(SPEC_FROM_FILE.getValue())) {
+            rawRetrieval = new FileSystemDataflowRetrieval();
+        } else {
+            rawRetrieval = new RegistryDataflowRetrieval(getLogger());
+        }
+
+        final DataflowRetrieval cachedRetrieval = new CachingDataflowRetrieval(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) {

Review comment:
       IOException is the only checked Exception. That's all that I was intending to catch here. 

##########
File path: nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/ExecuteStateless.java
##########
@@ -0,0 +1,1202 @@
+/*
+ * 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 com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.module.jaxb.JaxbAnnotationIntrospector;
+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.ComponentLog;
+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.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.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 javax.net.ssl.SSLContext;
+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 {
+    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");
+    static final AllowableValue SPEC_FROM_REGISTRY = new AllowableValue("Use NiFi Registry", "Use NiFi Registry", "Dataflow to run is stored in NiFi Registry");
+
+    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.");
+    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.");
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    static final PropertyDescriptor DATAFLOW_TIMEOUT = new Builder()
+        .name("Dataflow Timeout")
+        .displayName("Dataflow Timeout")
+        .description("Specifies the maximum amount of time for an invocation of the stateless flow to complete. If the flow does not complete within this amount of time, the incoming FlowFile, if " +
+            "any, will be routed to the timeout relationship. In any case, the dataflow will be canceled and the invocation will end")
+        .required(true)
+        .addValidator(TIME_PERIOD_VALIDATOR)
+        .expressionLanguageSupported(FLOWFILE_ATTRIBUTES)
+        .defaultValue("60 sec")
+        .build();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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 temporal 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();
+
+    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 DataflowRetrieval rawRetrieval;
+        if (specificationStrategy.equalsIgnoreCase(SPEC_FROM_FILE.getValue())) {
+            rawRetrieval = new FileSystemDataflowRetrieval();
+        } else {
+            rawRetrieval = new RegistryDataflowRetrieval(getLogger());
+        }
+
+        final DataflowRetrieval cachedRetrieval = new CachingDataflowRetrieval(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 {
+                final LogLevel logLevel = LogLevel.valueOf(bulletin.getLevel());
+                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 ignored) {
+                // There should be no malformed bulletin, but just in case the bulletin level, etc. ends up being null, catch the exception and ignore it,
+                // since there's not much we can do about it.

Review comment:
       Yeah, this should never happen. If it does, i'll actually log a warning.

##########
File path: nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/ExecuteStateless.java
##########
@@ -0,0 +1,1202 @@
+/*
+ * 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 com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.module.jaxb.JaxbAnnotationIntrospector;
+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.ComponentLog;
+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.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.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 javax.net.ssl.SSLContext;
+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 {
+    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");
+    static final AllowableValue SPEC_FROM_REGISTRY = new AllowableValue("Use NiFi Registry", "Use NiFi Registry", "Dataflow to run is stored in NiFi Registry");
+
+    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.");
+    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.");
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    static final PropertyDescriptor DATAFLOW_TIMEOUT = new Builder()
+        .name("Dataflow Timeout")
+        .displayName("Dataflow Timeout")
+        .description("Specifies the maximum amount of time for an invocation of the stateless flow to complete. If the flow does not complete within this amount of time, the incoming FlowFile, if " +
+            "any, will be routed to the timeout relationship. In any case, the dataflow will be canceled and the invocation will end")
+        .required(true)
+        .addValidator(TIME_PERIOD_VALIDATOR)
+        .expressionLanguageSupported(FLOWFILE_ATTRIBUTES)
+        .defaultValue("60 sec")
+        .build();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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 temporal 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();
+
+    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 DataflowRetrieval rawRetrieval;
+        if (specificationStrategy.equalsIgnoreCase(SPEC_FROM_FILE.getValue())) {
+            rawRetrieval = new FileSystemDataflowRetrieval();
+        } else {
+            rawRetrieval = new RegistryDataflowRetrieval(getLogger());
+        }
+
+        final DataflowRetrieval cachedRetrieval = new CachingDataflowRetrieval(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 {
+                final LogLevel logLevel = LogLevel.valueOf(bulletin.getLevel());
+                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 ignored) {
+                // There should be no malformed bulletin, but just in case the bulletin level, etc. ends up being null, catch the exception and ignore it,
+                // since there's not much we can do about it.
+            }
+        }
+    }
+
+    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, "flowfile-content");

Review comment:
       @exceptionfactory multiple invocations of the same processor should use the same directory. Invocations from different processors should not. This is why the next line here create a new File that is a child of 'flowfile-content' and whose name is the identifier of the Processor. This allows us to separate out content repos per processor - and even per thread, actually, as multiple concurrent threads cannot be writing to the same content repo. I could add in  'execute-stateless' to the filename.

##########
File path: nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/ExecuteStateless.java
##########
@@ -0,0 +1,1202 @@
+/*
+ * 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 com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.module.jaxb.JaxbAnnotationIntrospector;
+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.ComponentLog;
+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.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.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 javax.net.ssl.SSLContext;
+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 {
+    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");
+    static final AllowableValue SPEC_FROM_REGISTRY = new AllowableValue("Use NiFi Registry", "Use NiFi Registry", "Dataflow to run is stored in NiFi Registry");
+
+    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.");
+    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.");
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    static final PropertyDescriptor DATAFLOW_TIMEOUT = new Builder()
+        .name("Dataflow Timeout")
+        .displayName("Dataflow Timeout")
+        .description("Specifies the maximum amount of time for an invocation of the stateless flow to complete. If the flow does not complete within this amount of time, the incoming FlowFile, if " +
+            "any, will be routed to the timeout relationship. In any case, the dataflow will be canceled and the invocation will end")
+        .required(true)
+        .addValidator(TIME_PERIOD_VALIDATOR)
+        .expressionLanguageSupported(FLOWFILE_ATTRIBUTES)
+        .defaultValue("60 sec")
+        .build();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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 temporal 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();
+
+    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 DataflowRetrieval rawRetrieval;
+        if (specificationStrategy.equalsIgnoreCase(SPEC_FROM_FILE.getValue())) {
+            rawRetrieval = new FileSystemDataflowRetrieval();
+        } else {
+            rawRetrieval = new RegistryDataflowRetrieval(getLogger());
+        }
+
+        final DataflowRetrieval cachedRetrieval = new CachingDataflowRetrieval(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 {
+                final LogLevel logLevel = LogLevel.valueOf(bulletin.getLevel());
+                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 ignored) {
+                // There should be no malformed bulletin, but just in case the bulletin level, etc. ends up being null, catch the exception and ignore it,
+                // since there's not much we can do about it.
+            }
+        }
+    }
+
+    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, "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 interface DataflowRetrieval {

Review comment:
       Fair enough.

##########
File path: nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/ExecuteStateless.java
##########
@@ -0,0 +1,1202 @@
+/*
+ * 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 com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.module.jaxb.JaxbAnnotationIntrospector;
+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.ComponentLog;
+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.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.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 javax.net.ssl.SSLContext;
+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 {
+    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");
+    static final AllowableValue SPEC_FROM_REGISTRY = new AllowableValue("Use NiFi Registry", "Use NiFi Registry", "Dataflow to run is stored in NiFi Registry");
+
+    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.");
+    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.");
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    static final PropertyDescriptor DATAFLOW_TIMEOUT = new Builder()
+        .name("Dataflow Timeout")
+        .displayName("Dataflow Timeout")
+        .description("Specifies the maximum amount of time for an invocation of the stateless flow to complete. If the flow does not complete within this amount of time, the incoming FlowFile, if " +
+            "any, will be routed to the timeout relationship. In any case, the dataflow will be canceled and the invocation will end")
+        .required(true)
+        .addValidator(TIME_PERIOD_VALIDATOR)
+        .expressionLanguageSupported(FLOWFILE_ATTRIBUTES)
+        .defaultValue("60 sec")
+        .build();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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 temporal 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();
+
+    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 DataflowRetrieval rawRetrieval;
+        if (specificationStrategy.equalsIgnoreCase(SPEC_FROM_FILE.getValue())) {
+            rawRetrieval = new FileSystemDataflowRetrieval();
+        } else {
+            rawRetrieval = new RegistryDataflowRetrieval(getLogger());
+        }
+
+        final DataflowRetrieval cachedRetrieval = new CachingDataflowRetrieval(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 {
+                final LogLevel logLevel = LogLevel.valueOf(bulletin.getLevel());
+                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 ignored) {
+                // There should be no malformed bulletin, but just in case the bulletin level, etc. ends up being null, catch the exception and ignore it,
+                // since there's not much we can do about it.
+            }
+        }
+    }
+
+    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, "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 interface DataflowRetrieval {

Review comment:
       I would avoid controller services in this situation. The advantage to using a controller service would be that you could configure it once and reuse it many times for many instances of ExecuteStateless. Except that for registry you'd still need a different bucket, flow, version every time. All you'd really be able to reuse is the timeout, url, and maybe SSL Context. So the benefits of a controller service here, IMO, are outweighed but the complexity of having to configure that service.

##########
File path: nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafka_2_6.java
##########
@@ -493,6 +517,82 @@ public void process(final InputStream rawIn) throws IOException {
         }
     }
 
+    private List<FlowFile> pollFlowFiles(final ProcessContext context, final ProcessSession session) {

Review comment:
       Yes, good call. Can pull a few methods into a Publisher util class.

##########
File path: nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/ExecuteStateless.java
##########
@@ -0,0 +1,1202 @@
+/*
+ * 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 com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.module.jaxb.JaxbAnnotationIntrospector;
+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.ComponentLog;
+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.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.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 javax.net.ssl.SSLContext;
+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 {
+    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");
+    static final AllowableValue SPEC_FROM_REGISTRY = new AllowableValue("Use NiFi Registry", "Use NiFi Registry", "Dataflow to run is stored in NiFi Registry");
+
+    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.");
+    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.");
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    static final PropertyDescriptor DATAFLOW_TIMEOUT = new Builder()
+        .name("Dataflow Timeout")
+        .displayName("Dataflow Timeout")
+        .description("Specifies the maximum amount of time for an invocation of the stateless flow to complete. If the flow does not complete within this amount of time, the incoming FlowFile, if " +
+            "any, will be routed to the timeout relationship. In any case, the dataflow will be canceled and the invocation will end")
+        .required(true)
+        .addValidator(TIME_PERIOD_VALIDATOR)
+        .expressionLanguageSupported(FLOWFILE_ATTRIBUTES)
+        .defaultValue("60 sec")
+        .build();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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 temporal 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();
+
+    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.")

Review comment:
       I would say no. Adding another relationship means one more thing that the user has to configure. And that's really just a failsafe to avoid throwing Out Of Memory Errors. For the 90% use case, I think the user doesn't care to separate those two particular ideas of "rejected due to size" vs. "failed to process." For the other 10% where the user does care about this differentiation, they can easily enough add in a RouteOnAttribute to just filter out any FlowFile that exceeds the limit.

##########
File path: nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/ExecuteStateless.java
##########
@@ -0,0 +1,1202 @@
+/*
+ * 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 com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.module.jaxb.JaxbAnnotationIntrospector;
+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.ComponentLog;
+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.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.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 javax.net.ssl.SSLContext;
+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 {
+    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");
+    static final AllowableValue SPEC_FROM_REGISTRY = new AllowableValue("Use NiFi Registry", "Use NiFi Registry", "Dataflow to run is stored in NiFi Registry");
+
+    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.");
+    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.");
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    static final PropertyDescriptor DATAFLOW_TIMEOUT = new Builder()
+        .name("Dataflow Timeout")
+        .displayName("Dataflow Timeout")
+        .description("Specifies the maximum amount of time for an invocation of the stateless flow to complete. If the flow does not complete within this amount of time, the incoming FlowFile, if " +
+            "any, will be routed to the timeout relationship. In any case, the dataflow will be canceled and the invocation will end")
+        .required(true)
+        .addValidator(TIME_PERIOD_VALIDATOR)
+        .expressionLanguageSupported(FLOWFILE_ATTRIBUTES)
+        .defaultValue("60 sec")
+        .build();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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 temporal 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();
+
+    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 DataflowRetrieval rawRetrieval;
+        if (specificationStrategy.equalsIgnoreCase(SPEC_FROM_FILE.getValue())) {
+            rawRetrieval = new FileSystemDataflowRetrieval();
+        } else {
+            rawRetrieval = new RegistryDataflowRetrieval(getLogger());
+        }
+
+        final DataflowRetrieval cachedRetrieval = new CachingDataflowRetrieval(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 {
+                final LogLevel logLevel = LogLevel.valueOf(bulletin.getLevel());
+                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 ignored) {
+                // There should be no malformed bulletin, but just in case the bulletin level, etc. ends up being null, catch the exception and ignore it,
+                // since there's not much we can do about it.
+            }
+        }
+    }
+
+    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, "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) {

Review comment:
       Yeah while testing & debugging, I saw things in the logs indicating Processor ID, etc. and I couldn't search based on that. So this will be helpful when seeing something in the logs and wanting to see how that processor is used in the flow.

##########
File path: nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/ExecuteStateless.java
##########
@@ -0,0 +1,1202 @@
+/*
+ * 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 com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.module.jaxb.JaxbAnnotationIntrospector;
+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.ComponentLog;
+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.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.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 javax.net.ssl.SSLContext;
+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 {
+    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");
+    static final AllowableValue SPEC_FROM_REGISTRY = new AllowableValue("Use NiFi Registry", "Use NiFi Registry", "Dataflow to run is stored in NiFi Registry");
+
+    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.");
+    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.");
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    static final PropertyDescriptor DATAFLOW_TIMEOUT = new Builder()
+        .name("Dataflow Timeout")
+        .displayName("Dataflow Timeout")
+        .description("Specifies the maximum amount of time for an invocation of the stateless flow to complete. If the flow does not complete within this amount of time, the incoming FlowFile, if " +
+            "any, will be routed to the timeout relationship. In any case, the dataflow will be canceled and the invocation will end")
+        .required(true)
+        .addValidator(TIME_PERIOD_VALIDATOR)
+        .expressionLanguageSupported(FLOWFILE_ATTRIBUTES)
+        .defaultValue("60 sec")
+        .build();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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 temporal 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();
+
+    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 DataflowRetrieval rawRetrieval;
+        if (specificationStrategy.equalsIgnoreCase(SPEC_FROM_FILE.getValue())) {
+            rawRetrieval = new FileSystemDataflowRetrieval();
+        } else {
+            rawRetrieval = new RegistryDataflowRetrieval(getLogger());
+        }
+
+        final DataflowRetrieval cachedRetrieval = new CachingDataflowRetrieval(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();

Review comment:
       Thanks. You're right. Those were not properly surfacing. I had to make a few adjustments.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] exceptionfactory commented on a change in pull request #5412: NIFI-9239: Updated Consume/Publish Kafka processors to support Exactl…

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on a change in pull request #5412:
URL: https://github.com/apache/nifi/pull/5412#discussion_r716975249



##########
File path: nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/ExecuteStateless.java
##########
@@ -0,0 +1,1202 @@
+/*
+ * 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 com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.module.jaxb.JaxbAnnotationIntrospector;
+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.ComponentLog;
+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.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.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 javax.net.ssl.SSLContext;
+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 {
+    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");
+    static final AllowableValue SPEC_FROM_REGISTRY = new AllowableValue("Use NiFi Registry", "Use NiFi Registry", "Dataflow to run is stored in NiFi Registry");
+
+    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.");
+    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.");
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    static final PropertyDescriptor DATAFLOW_TIMEOUT = new Builder()
+        .name("Dataflow Timeout")
+        .displayName("Dataflow Timeout")
+        .description("Specifies the maximum amount of time for an invocation of the stateless flow to complete. If the flow does not complete within this amount of time, the incoming FlowFile, if " +
+            "any, will be routed to the timeout relationship. In any case, the dataflow will be canceled and the invocation will end")
+        .required(true)
+        .addValidator(TIME_PERIOD_VALIDATOR)
+        .expressionLanguageSupported(FLOWFILE_ATTRIBUTES)
+        .defaultValue("60 sec")
+        .build();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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 temporal 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();
+
+    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 DataflowRetrieval rawRetrieval;
+        if (specificationStrategy.equalsIgnoreCase(SPEC_FROM_FILE.getValue())) {
+            rawRetrieval = new FileSystemDataflowRetrieval();
+        } else {
+            rawRetrieval = new RegistryDataflowRetrieval(getLogger());
+        }
+
+        final DataflowRetrieval cachedRetrieval = new CachingDataflowRetrieval(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 {
+                final LogLevel logLevel = LogLevel.valueOf(bulletin.getLevel());
+                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 ignored) {
+                // There should be no malformed bulletin, but just in case the bulletin level, etc. ends up being null, catch the exception and ignore it,
+                // since there's not much we can do about it.
+            }
+        }
+    }
+
+    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, "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 interface DataflowRetrieval {

Review comment:
       On further consideration, what do you think about turning this into a Controller Service interface?  File-based and Registry-based implementations should cover most use cases, but that would allow for custom implementations using other approaches.  That would also remove the direct dependency on nifi-registry, and also remove the Registry SSL Context Service property.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] markap14 commented on a change in pull request #5412: NIFI-9239: Updated Consume/Publish Kafka processors to support Exactl…

Posted by GitBox <gi...@apache.org>.
markap14 commented on a change in pull request #5412:
URL: https://github.com/apache/nifi/pull/5412#discussion_r717718403



##########
File path: nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/ExecuteStateless.java
##########
@@ -0,0 +1,1202 @@
+/*
+ * 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 com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.module.jaxb.JaxbAnnotationIntrospector;
+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.ComponentLog;
+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.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.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 javax.net.ssl.SSLContext;
+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 {
+    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");
+    static final AllowableValue SPEC_FROM_REGISTRY = new AllowableValue("Use NiFi Registry", "Use NiFi Registry", "Dataflow to run is stored in NiFi Registry");
+
+    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.");
+    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.");
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    static final PropertyDescriptor DATAFLOW_TIMEOUT = new Builder()
+        .name("Dataflow Timeout")
+        .displayName("Dataflow Timeout")
+        .description("Specifies the maximum amount of time for an invocation of the stateless flow to complete. If the flow does not complete within this amount of time, the incoming FlowFile, if " +
+            "any, will be routed to the timeout relationship. In any case, the dataflow will be canceled and the invocation will end")
+        .required(true)
+        .addValidator(TIME_PERIOD_VALIDATOR)
+        .expressionLanguageSupported(FLOWFILE_ATTRIBUTES)
+        .defaultValue("60 sec")
+        .build();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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 temporal 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();
+
+    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.")

Review comment:
       I would say no. Adding another relationship means one more thing that the user has to configure. And that's really just a failsafe to avoid throwing Out Of Memory Errors. For the 90% use case, I think the user doesn't care to separate those two particular ideas of "rejected due to size" vs. "failed to process." For the other 10% where the user does care about this differentiation, they can easily enough add in a RouteOnAttribute to just filter out any FlowFile that exceeds the limit.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] markap14 commented on a change in pull request #5412: NIFI-9239: Updated Consume/Publish Kafka processors to support Exactl…

Posted by GitBox <gi...@apache.org>.
markap14 commented on a change in pull request #5412:
URL: https://github.com/apache/nifi/pull/5412#discussion_r716934722



##########
File path: nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/ExecuteStateless.java
##########
@@ -0,0 +1,1202 @@
+/*
+ * 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 com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.module.jaxb.JaxbAnnotationIntrospector;
+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.ComponentLog;
+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.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.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 javax.net.ssl.SSLContext;
+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 {
+    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");
+    static final AllowableValue SPEC_FROM_REGISTRY = new AllowableValue("Use NiFi Registry", "Use NiFi Registry", "Dataflow to run is stored in NiFi Registry");
+
+    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.");
+    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.");
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    static final PropertyDescriptor DATAFLOW_TIMEOUT = new Builder()
+        .name("Dataflow Timeout")
+        .displayName("Dataflow Timeout")
+        .description("Specifies the maximum amount of time for an invocation of the stateless flow to complete. If the flow does not complete within this amount of time, the incoming FlowFile, if " +
+            "any, will be routed to the timeout relationship. In any case, the dataflow will be canceled and the invocation will end")
+        .required(true)
+        .addValidator(TIME_PERIOD_VALIDATOR)
+        .expressionLanguageSupported(FLOWFILE_ATTRIBUTES)
+        .defaultValue("60 sec")
+        .build();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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();
+
+    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 temporal 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();
+
+    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 DataflowRetrieval rawRetrieval;
+        if (specificationStrategy.equalsIgnoreCase(SPEC_FROM_FILE.getValue())) {
+            rawRetrieval = new FileSystemDataflowRetrieval();
+        } else {
+            rawRetrieval = new RegistryDataflowRetrieval(getLogger());
+        }
+
+        final DataflowRetrieval cachedRetrieval = new CachingDataflowRetrieval(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);

Review comment:
       I like the idea. But when the flow is saved via NiFi's  Download Flow Definition, what we get is basically an empty VersionedFlowSnapshot wrapper, with only the "flowContents" populated. So the name of the flow, etc., are not available.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] markap14 commented on a change in pull request #5412: NIFI-9239: Updated Consume/Publish Kafka processors to support Exactl…

Posted by GitBox <gi...@apache.org>.
markap14 commented on a change in pull request #5412:
URL: https://github.com/apache/nifi/pull/5412#discussion_r717715814



##########
File path: nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-2-6-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafka_2_6.java
##########
@@ -493,6 +517,82 @@ public void process(final InputStream rawIn) throws IOException {
         }
     }
 
+    private List<FlowFile> pollFlowFiles(final ProcessContext context, final ProcessSession session) {

Review comment:
       Yes, good call. Can pull a few methods into a Publisher util class.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] exceptionfactory commented on pull request #5412: NIFI-9239: Updated Consume/Publish Kafka processors to support Exactl…

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on pull request #5412:
URL: https://github.com/apache/nifi/pull/5412#issuecomment-932525786


   Thanks for making the updates @markap14, I will take another look through this for runtime verification soon.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org