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

[nifi] branch main updated (151a936 -> be2b80d)

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

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


    from 151a936  NIFI-9288: Updated data model to use for cnofig verification requests so that only properties are necessary not the full component/config dtos. Also added endpoint necessary for determining which attributes are referenced by a component's properties and removed the referencedAttributes field from the compnoents themselves, since there's now a new endpoint for it. Also fixed a bug that was encountered where the VerifyConfigRequestDTO's complete flag was incorrect in case  [...]
     new d722b82  NIFI-9248 Use hive-exec:core instead of the "regular" hive-exec dependency
     new be2b80d  NIFI-9292: Added getInputPortNames() and getOutputPortNames() to StatelessDataflowDefinition and getReadOnlyExtensionsDirectories() to StatelessEngineConfiguration - Addressed issue identified in system test where a flow can fail due to FailurePortEncounteredException but then purge the flow in the background after the call to trigger completes

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../kafka/connect/StatelessKafkaConnectorUtil.java |  6 ++
 .../nar/StandardExtensionDiscoveringManager.java   |  6 ++
 .../nifi-hive-bundle/nifi-hive3-processors/pom.xml | 83 ++++++++++++++++++++++
 .../processors/stateless/ExecuteStateless.java     | 15 ++++
 .../PropertiesFileEngineConfigurationParser.java   | 21 ++++++
 .../engine/StatelessEngineConfiguration.java       | 36 ++++++++++
 .../nifi/stateless/flow/DataflowDefinition.java    |  4 ++
 ...ropertiesFileEngineConfigurationParserTest.java | 29 ++++++++
 nifi-stateless/nifi-stateless-assembly/README.md   |  9 ++-
 .../stateless/bootstrap/StatelessBootstrap.java    |  6 ++
 .../nifi/extensions/ExtensionRepository.java       |  2 +
 .../extensions/FileSystemExtensionRepository.java  | 62 +++++++++++-----
 .../nifi/stateless/engine/FlowPurgeAction.java     |  8 +--
 .../engine/StandardExecutionProgress.java          |  8 ++-
 .../stateless/engine/StandardStatelessEngine.java  | 34 ++++++---
 .../stateless/flow/StandardDataflowDefinition.java | 18 ++++-
 .../flow/StandardStatelessDataflowFactory.java     |  5 +-
 .../nifi/stateless/flow/StandardStatelessFlow.java |  4 +-
 .../TestPropertiesFileFlowDefinitionParser.java    |  6 ++
 .../apache/nifi/stateless/StatelessSystemIT.java   | 22 ++++++
 20 files changed, 343 insertions(+), 41 deletions(-)
 copy nifi-api/src/main/java/org/apache/nifi/reporting/Severity.java => nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/engine/FlowPurgeAction.java (89%)

[nifi] 01/02: NIFI-9248 Use hive-exec:core instead of the "regular" hive-exec dependency

Posted by jg...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit d722b820a18836b13df5fb5d85b62f07e5979720
Author: Denes Arvay <de...@apache.org>
AuthorDate: Thu Sep 9 21:36:56 2021 +0200

    NIFI-9248 Use hive-exec:core instead of the "regular" hive-exec dependency
    
    - Excluded org.apache.hive:hive-exec and added the "core" classifier version
    - Added the removed dependencies explicitly
    
    NIFI-9248 Add additional log4j excludes
    
    Signed-off-by: Matthew Burgess <ma...@apache.org>
    
    This closes #5414
---
 .../nifi-hive-bundle/nifi-hive3-processors/pom.xml | 83 ++++++++++++++++++++++
 1 file changed, 83 insertions(+)

diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/pom.xml b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/pom.xml
index bb0b10d..8535f88 100644
--- a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/pom.xml
@@ -99,6 +99,81 @@
         </dependency>
         <dependency>
             <groupId>org.apache.hive</groupId>
+            <artifactId>hive-exec</artifactId>
+            <version>${hive.version}</version>
+            <classifier>core</classifier>
+            <exclusions>
+                <exclusion>
+                    <groupId>log4j</groupId>
+                    <artifactId>log4j</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+
+        <!-- hive-exec:core doesn't contain these dependencies (as opposed to regular hive-exec) so these need to be pulled in -->
+        <dependency>
+            <groupId>com.esotericsoftware</groupId>
+            <artifactId>kryo-shaded</artifactId>
+            <version>3.0.3</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.commons</groupId>
+            <artifactId>commons-lang3</artifactId>
+            <version>3.2</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.avro</groupId>
+            <artifactId>avro-mapred</artifactId>
+            <classifier>hadoop2</classifier>
+            <version>1.8.1</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.mortbay.jetty</groupId>
+                    <artifactId>servlet-api</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>com.googlecode.javaewah</groupId>
+            <artifactId>JavaEWAH</artifactId>
+            <version>0.3.2</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hive</groupId>
+            <artifactId>hive-spark-client</artifactId>
+            <version>${hive.version}</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>log4j</groupId>
+                    <artifactId>log4j</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.orc</groupId>
+            <artifactId>orc-tools</artifactId>
+            <version>1.5.6</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>log4j</groupId>
+                    <artifactId>log4j</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.jodd</groupId>
+            <artifactId>jodd-core</artifactId>
+            <version>3.5.2</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hive</groupId>
+            <artifactId>hive-storage-api</artifactId>
+            <version>2.7.0</version>
+        </dependency>
+        <!-- /hive-exec:core doesn't contain these dependencies (as opposed to regular hive-exec) so these need to be pulled in -->
+
+        <dependency>
+            <groupId>org.apache.hive</groupId>
             <artifactId>hive-streaming</artifactId>
             <version>${hive.version}</version>
             <exclusions>
@@ -110,6 +185,10 @@
                     <groupId>log4j</groupId>
                     <artifactId>log4j</artifactId>
                 </exclusion>
+                <exclusion>
+                    <groupId>org.apache.hive</groupId>
+                    <artifactId>hive-exec</artifactId>
+                </exclusion>
             </exclusions>
         </dependency>
         <dependency>
@@ -121,6 +200,10 @@
                     <groupId>log4j</groupId>
                     <artifactId>log4j</artifactId>
                 </exclusion>
+                <exclusion>
+                    <groupId>org.apache.hive</groupId>
+                    <artifactId>hive-exec</artifactId>
+                </exclusion>
             </exclusions>
         </dependency>
         <!-- Override groovy-all:2.4.11 from Hive -->

[nifi] 02/02: NIFI-9292: Added getInputPortNames() and getOutputPortNames() to StatelessDataflowDefinition and getReadOnlyExtensionsDirectories() to StatelessEngineConfiguration - Addressed issue identified in system test where a flow can fail due to FailurePortEncounteredException but then purge the flow in the background after the call to trigger completes

Posted by jg...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit be2b80d9094beeb3a2fa2b6a0ecbee2eb8b0935d
Author: Mark Payne <ma...@hotmail.com>
AuthorDate: Thu Oct 7 10:38:47 2021 -0400

    NIFI-9292: Added getInputPortNames() and getOutputPortNames() to StatelessDataflowDefinition and getReadOnlyExtensionsDirectories() to StatelessEngineConfiguration
    - Addressed issue identified in system test where a flow can fail due to FailurePortEncounteredException but then purge the flow in the background after the call to trigger completes
    
    Signed-off-by: Joe Gresock <jg...@gmail.com>
    
    This closes #5450.
---
 .../kafka/connect/StatelessKafkaConnectorUtil.java |  6 +++
 .../nar/StandardExtensionDiscoveringManager.java   |  6 +++
 .../processors/stateless/ExecuteStateless.java     | 15 ++++++
 .../PropertiesFileEngineConfigurationParser.java   | 21 ++++++++
 .../engine/StatelessEngineConfiguration.java       | 36 +++++++++++++
 .../nifi/stateless/flow/DataflowDefinition.java    |  4 ++
 ...ropertiesFileEngineConfigurationParserTest.java | 29 ++++++++++
 nifi-stateless/nifi-stateless-assembly/README.md   |  9 +++-
 .../stateless/bootstrap/StatelessBootstrap.java    |  6 +++
 .../nifi/extensions/ExtensionRepository.java       |  2 +
 .../extensions/FileSystemExtensionRepository.java  | 62 +++++++++++++++-------
 .../engine/FlowPurgeAction.java}                   | 16 ++----
 .../engine/StandardExecutionProgress.java          |  8 +--
 .../stateless/engine/StandardStatelessEngine.java  | 34 ++++++++----
 .../stateless/flow/StandardDataflowDefinition.java | 18 ++++++-
 .../flow/StandardStatelessDataflowFactory.java     |  5 +-
 .../nifi/stateless/flow/StandardStatelessFlow.java |  4 +-
 .../TestPropertiesFileFlowDefinitionParser.java    |  6 +++
 .../apache/nifi/stateless/StatelessSystemIT.java   | 22 ++++++++
 19 files changed, 260 insertions(+), 49 deletions(-)

diff --git a/nifi-external/nifi-kafka-connect/nifi-kafka-connector/src/main/java/org/apache/nifi/kafka/connect/StatelessKafkaConnectorUtil.java b/nifi-external/nifi-kafka-connect/nifi-kafka-connector/src/main/java/org/apache/nifi/kafka/connect/StatelessKafkaConnectorUtil.java
index 2dc76eb..2406e7a 100644
--- a/nifi-external/nifi-kafka-connect/nifi-kafka-connector/src/main/java/org/apache/nifi/kafka/connect/StatelessKafkaConnectorUtil.java
+++ b/nifi-external/nifi-kafka-connect/nifi-kafka-connector/src/main/java/org/apache/nifi/kafka/connect/StatelessKafkaConnectorUtil.java
@@ -38,6 +38,7 @@ import java.net.URL;
 import java.net.URLClassLoader;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
@@ -274,6 +275,11 @@ public class StatelessKafkaConnectorUtil {
             }
 
             @Override
+            public Collection<File> getReadOnlyExtensionsDirectories() {
+                return Collections.emptyList();
+            }
+
+            @Override
             public File getKrb5File() {
                 return new File(properties.getOrDefault(KRB5_FILE, DEFAULT_KRB5_FILE));
             }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/StandardExtensionDiscoveringManager.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/StandardExtensionDiscoveringManager.java
index 09031a8..6a78a9c 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/StandardExtensionDiscoveringManager.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/StandardExtensionDiscoveringManager.java
@@ -568,6 +568,12 @@ public class StandardExtensionDiscoveringManager implements ExtensionDiscovering
             return tempComponent;
         } catch (final Exception e) {
             logger.error("Could not instantiate class of type {} using ClassLoader for bundle {}", classType, bundleCoordinate, e);
+            if (logger.isDebugEnabled() && bundleClassLoader instanceof URLClassLoader) {
+                final URLClassLoader urlClassLoader = (URLClassLoader) bundleClassLoader;
+                final List<URL> availableUrls = Arrays.asList(urlClassLoader.getURLs());
+                logger.debug("Available URLs for Bundle ClassLoader {}: {}", bundleCoordinate, availableUrls);
+            }
+
             return null;
         }
     }
diff --git a/nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/ExecuteStateless.java b/nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/ExecuteStateless.java
index cc06f1d..519c954 100644
--- a/nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/ExecuteStateless.java
+++ b/nifi-nar-bundles/nifi-stateless-processor-bundle/nifi-stateless-processor/src/main/java/org/apache/nifi/processors/stateless/ExecuteStateless.java
@@ -788,6 +788,16 @@ public class ExecuteStateless extends AbstractProcessor implements Searchable {
             }
 
             @Override
+            public Set<String> getInputPortNames() {
+                return Collections.emptySet();
+            }
+
+            @Override
+            public Set<String> getOutputPortNames() {
+                return failurePortNames;
+            }
+
+            @Override
             public List<ParameterContextDefinition> getParameterContexts() {
                 return null;
             }
@@ -857,6 +867,11 @@ public class ExecuteStateless extends AbstractProcessor implements Searchable {
             }
 
             @Override
+            public Collection<File> getReadOnlyExtensionsDirectories() {
+                return Collections.emptyList();
+            }
+
+            @Override
             public File getKrb5File() {
                 return krb5Conf;
             }
diff --git a/nifi-stateless/nifi-stateless-api/src/main/java/org/apache/nifi/stateless/config/PropertiesFileEngineConfigurationParser.java b/nifi-stateless/nifi-stateless-api/src/main/java/org/apache/nifi/stateless/config/PropertiesFileEngineConfigurationParser.java
index 19eb7d7..525a88d 100644
--- a/nifi-stateless/nifi-stateless-api/src/main/java/org/apache/nifi/stateless/config/PropertiesFileEngineConfigurationParser.java
+++ b/nifi-stateless/nifi-stateless-api/src/main/java/org/apache/nifi/stateless/config/PropertiesFileEngineConfigurationParser.java
@@ -32,6 +32,7 @@ import java.io.UncheckedIOException;
 import java.security.SecureRandom;
 import java.util.ArrayList;
 import java.util.Base64;
+import java.util.Collection;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
@@ -39,6 +40,7 @@ import java.util.Optional;
 import java.util.Properties;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
+import java.util.stream.Collectors;
 
 public class PropertiesFileEngineConfigurationParser {
     private static final Logger logger = LoggerFactory.getLogger(PropertiesFileEngineConfigurationParser.class);
@@ -46,6 +48,7 @@ public class PropertiesFileEngineConfigurationParser {
 
     private static final String NAR_DIRECTORY = PREFIX + "nar.directory";
     private static final String EXTENSIONS_DIRECTORY = PREFIX + "extensions.directory";
+    private static final String READONLY_EXTENSIONS_DIRECTORY = PREFIX + "readonly.extensions.directory.";
     private static final String WORKING_DIRECTORY = PREFIX + "working.directory";
     private static final String CONTENT_REPO_DIRECTORY = PREFIX + "content.repository.directory";
 
@@ -92,6 +95,8 @@ public class PropertiesFileEngineConfigurationParser {
             throw new StatelessConfigurationException("Extensions Directory " + narDirectory.getAbsolutePath() + " specified in properties file does not exist and could not be created");
         }
 
+        final List<File> readOnlyExtensionsDirectories = getReadOnlyExtensionsDirectories(properties);
+
         final String contentRepoDirectoryFilename = properties.getProperty(CONTENT_REPO_DIRECTORY, "");
         final File contentRepoDirectory = contentRepoDirectoryFilename.isEmpty() ? null : new File(contentRepoDirectoryFilename);
 
@@ -120,6 +125,11 @@ public class PropertiesFileEngineConfigurationParser {
             }
 
             @Override
+            public Collection<File> getReadOnlyExtensionsDirectories() {
+                return readOnlyExtensionsDirectories;
+            }
+
+            @Override
             public File getKrb5File() {
                 return krb5File;
             }
@@ -146,6 +156,17 @@ public class PropertiesFileEngineConfigurationParser {
         };
     }
 
+
+    private List<File> getReadOnlyExtensionsDirectories(final Properties properties) {
+        return properties.keySet().stream()
+            .map(Object::toString)
+            .filter(key -> key.startsWith(READONLY_EXTENSIONS_DIRECTORY))
+            .map(properties::getProperty)
+            .map(File::new)
+            .collect(Collectors.toList());
+    }
+
+
     private List<ExtensionClientDefinition> parseExtensionClients(final Properties properties) {
         final Map<String, ExtensionClientDefinition> extensionClientDefinitions = new LinkedHashMap<>();
 
diff --git a/nifi-stateless/nifi-stateless-api/src/main/java/org/apache/nifi/stateless/engine/StatelessEngineConfiguration.java b/nifi-stateless/nifi-stateless-api/src/main/java/org/apache/nifi/stateless/engine/StatelessEngineConfiguration.java
index 8f27c66..1f8b282 100644
--- a/nifi-stateless/nifi-stateless-api/src/main/java/org/apache/nifi/stateless/engine/StatelessEngineConfiguration.java
+++ b/nifi-stateless/nifi-stateless-api/src/main/java/org/apache/nifi/stateless/engine/StatelessEngineConfiguration.java
@@ -21,16 +21,36 @@ import org.apache.nifi.stateless.config.ExtensionClientDefinition;
 import org.apache.nifi.stateless.config.SslContextDefinition;
 
 import java.io.File;
+import java.util.Collection;
 import java.util.List;
 import java.util.Optional;
 
 public interface StatelessEngineConfiguration {
+    /**
+     * @return the directory to use for 'scratch space', such as unpacking NAR files
+     */
     File getWorkingDirectory();
 
+    /**
+     * @return the directory containing the NiFi Stateless NAR and all other necessary libraries required for the Stateless engine to be bootstrapped
+     */
     File getNarDirectory();
 
+    /**
+     * @return a directory containing extensions that should be loaded and into which extensions can be downloaded
+     */
     File getExtensionsDirectory();
 
+    /**
+     * @return a collection of zero or more directories that contain extensions that should be loaded by the stateless engine. The engine will not download
+     * any extensions into these directories or write to them but will read any NAR files that are found within these directories. The engine will not recurse
+     * into subdirectories of these directories.
+     */
+    Collection<File> getReadOnlyExtensionsDirectories();
+
+    /**
+     * @return the KRB5 file to use for establishing any Kerberos connections, or <code>null</code> if no KRB5 file is to be used
+     */
     File getKrb5File();
 
     /**
@@ -38,12 +58,28 @@ public interface StatelessEngineConfiguration {
      */
     Optional<File> getContentRepositoryDirectory();
 
+    /**
+     * @return the definition needed to create an SSL Context that can be used for interacting with a Nexus Repository or retrieving a flow from the Flow Registry, etc.
+     * This SSL Context will NOT be made available to extensions running in the dataflow.
+     */
     SslContextDefinition getSslContext();
 
+    /**
+     * @return a sensitive properties key that extensions may choose to use for encrypting/decrypting sensitive information
+     */
     String getSensitivePropsKey();
 
+    /**
+     * @return a List of definitions for Extension Clients. These clients will be used to attempt to download any extensions that are required by a dataflow that are not
+     * already present. The clients will be used in the order in which they are provided. Therefore, if one client is to be preferred over another, it should come first in
+     * the List.
+     */
     List<ExtensionClientDefinition> getExtensionClients();
 
+    /**
+     * When discovering extensions, indicates whether or not the discovered extensions should be logged at an INFO level.
+     * @return <code>true</code> if the discovered extensions should be logged at an INFO level, <code>false</code> if they should not be logged (or logged only at DEBUG level).
+     */
     default boolean isLogExtensionDiscovery() {
         return true;
     }
diff --git a/nifi-stateless/nifi-stateless-api/src/main/java/org/apache/nifi/stateless/flow/DataflowDefinition.java b/nifi-stateless/nifi-stateless-api/src/main/java/org/apache/nifi/stateless/flow/DataflowDefinition.java
index cdb8036..2432684 100644
--- a/nifi-stateless/nifi-stateless-api/src/main/java/org/apache/nifi/stateless/flow/DataflowDefinition.java
+++ b/nifi-stateless/nifi-stateless-api/src/main/java/org/apache/nifi/stateless/flow/DataflowDefinition.java
@@ -31,6 +31,10 @@ public interface DataflowDefinition<T> {
 
     Set<String> getFailurePortNames();
 
+    Set<String> getInputPortNames();
+
+    Set<String> getOutputPortNames();
+
     List<ParameterContextDefinition> getParameterContexts();
 
     List<ReportingTaskDefinition> getReportingTaskDefinitions();
diff --git a/nifi-stateless/nifi-stateless-api/src/test/java/org/apache/nifi/stateless/config/PropertiesFileEngineConfigurationParserTest.java b/nifi-stateless/nifi-stateless-api/src/test/java/org/apache/nifi/stateless/config/PropertiesFileEngineConfigurationParserTest.java
index a21fac2..5b134cc 100644
--- a/nifi-stateless/nifi-stateless-api/src/test/java/org/apache/nifi/stateless/config/PropertiesFileEngineConfigurationParserTest.java
+++ b/nifi-stateless/nifi-stateless-api/src/test/java/org/apache/nifi/stateless/config/PropertiesFileEngineConfigurationParserTest.java
@@ -28,8 +28,10 @@ import java.io.IOException;
 import java.io.OutputStream;
 import java.nio.file.Files;
 import java.nio.file.Path;
+import java.util.Collection;
 import java.util.Properties;
 
+import static org.junit.Assert.assertTrue;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertNotNull;
@@ -85,6 +87,33 @@ public class PropertiesFileEngineConfigurationParserTest {
         assertEquals(sensitivePropsKey, reloadedConfiguration.getSensitivePropsKey());
     }
 
+    @Test
+    public void testReadOnlyExtensionsDirectoriesParsed() throws IOException, StatelessConfigurationException {
+        final Properties properties = getRequiredProperties();
+        properties.setProperty("nifi.stateless.readonly.extensions.directory.abc", "target/1");
+        properties.setProperty("nifi.stateless.readonly.extensions.directory.xyz", "target/2");
+        final File propertiesFile = getPropertiesFile(properties);
+
+        final StatelessEngineConfiguration configuration = parser.parseEngineConfiguration(propertiesFile);
+        assertNotNull(configuration);
+        final Collection<File> readOnlyExtensionsDirs = configuration.getReadOnlyExtensionsDirectories();
+        assertEquals(2, readOnlyExtensionsDirs.size());
+        assertTrue(readOnlyExtensionsDirs.contains(new File("target/1")));
+        assertTrue(readOnlyExtensionsDirs.contains(new File("target/2")));
+    }
+
+    @Test
+    public void testReadOnlyExtensionsDirectoriesNotSpecified() throws IOException, StatelessConfigurationException {
+        final Properties properties = getRequiredProperties();
+        final File propertiesFile = getPropertiesFile(properties);
+
+        final StatelessEngineConfiguration configuration = parser.parseEngineConfiguration(propertiesFile);
+        assertNotNull(configuration);
+        final Collection<File> readOnlyExtensionsDirs = configuration.getReadOnlyExtensionsDirectories();
+        assertNotNull(readOnlyExtensionsDirs);
+        assertEquals(0, readOnlyExtensionsDirs.size());
+    }
+
     private Properties getRequiredProperties() {
         final Properties properties = new Properties();
 
diff --git a/nifi-stateless/nifi-stateless-assembly/README.md b/nifi-stateless/nifi-stateless-assembly/README.md
index 27ca853..9910dbe 100644
--- a/nifi-stateless/nifi-stateless-assembly/README.md
+++ b/nifi-stateless/nifi-stateless-assembly/README.md
@@ -211,6 +211,13 @@ The following properties may be used for configuring security parameters:
 | nifi.stateless.sensitive.props.key | The dataflow does not hold sensitive passwords, but some processors may have a need to encrypt data before storing it. This key is used to allow processors to encrypt and decrypt data. At present, the only Processor supported by the community that makes use of this feature is hte GetJMSTopic processor, which is deprecated. However, it is provided here for completeness. | Some Passphrase That's Difficult to Guess |
 | nifi.stateless.kerberos.krb5.file | The KRB5 file to use for interacting with Kerberos. This is only necessary if the dataflow interacts with a Kerberized data source/sink. If not specified, will default to `/etc/krb5.conf` | /etc/krb5.conf |
 
+A key tenant of NiFi is a separation between the framework and the extensions that operate on data. As such, we must have a mechanism for dictating where extensions can be located.
+The following properties may be used to indicate where extensions are to be located:
+
+| Property Name | Description | Example Value |
+|---------------|-------------|---------------|
+| nifi.stateless.extensions.directory | The directory that contains extensions that should be loaded. If extensions are to be downloaded (see below), they will be written to this directory, so it must be writable by the user who launches the application. | /var/lib/nifi/extensions |
+| nifi.stateless.readonly.extensions.directory.<suffix> | One or more directories may be specified as read-only extensions directories. Extensions will be loaded from these directories (but not their subdirectories), but these directories do not need to be writable. | /mnt/nifi-extensions/aws-extensions/ |
 
 When Stateless NiFi is started, it parses the provided dataflow and determines which bundles/extensions are necessary
 to run the dataflow. If an extension is not available, or the version referenced by the flow is not available, Stateless
@@ -222,7 +229,7 @@ the first `type` property refers to the same client as the first `baseUrl` prope
 `ABC`. Similarly, the second `type` and `baseUrl` properties refer to the same client because they have the same 'key':
 `XYZ`.
 
-Any extension that is downloaded will be stored in the directory specified by the `nifi.stateless.nar.directory` property described above.
+Any extension that is downloaded will be stored in the directory specified by the `nifi.stateless.extensions.directory` property described above.
 
 | Property Name | Description | Example Value |
 |---------------|-------------|---------------|
diff --git a/nifi-stateless/nifi-stateless-bootstrap/src/main/java/org/apache/nifi/stateless/bootstrap/StatelessBootstrap.java b/nifi-stateless/nifi-stateless-bootstrap/src/main/java/org/apache/nifi/stateless/bootstrap/StatelessBootstrap.java
index d99cea5..3f799b9 100644
--- a/nifi-stateless/nifi-stateless-bootstrap/src/main/java/org/apache/nifi/stateless/bootstrap/StatelessBootstrap.java
+++ b/nifi-stateless/nifi-stateless-bootstrap/src/main/java/org/apache/nifi/stateless/bootstrap/StatelessBootstrap.java
@@ -39,6 +39,7 @@ import java.net.URL;
 import java.net.URLClassLoader;
 import java.nio.file.Path;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
@@ -124,6 +125,11 @@ public class StatelessBootstrap {
             urls[i] = url;
         }
 
+        logger.info("Creating Stateless Bootstrap with the following URLs in the classpath: {}", Arrays.asList(urls));
+        if (rootClassLoader instanceof URLClassLoader) {
+            logger.info("Additionally, Root ClassLoader has the following URLs available: {}", Arrays.asList(((URLClassLoader) rootClassLoader).getURLs()));
+        }
+
         final URLClassLoader statelessClassLoader = new URLClassLoader(urls, rootClassLoader);
         Thread.currentThread().setContextClassLoader(statelessClassLoader);
         return new StatelessBootstrap(statelessClassLoader, engineConfiguration);
diff --git a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/extensions/ExtensionRepository.java b/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/extensions/ExtensionRepository.java
index 291a301..d86b565 100644
--- a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/extensions/ExtensionRepository.java
+++ b/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/extensions/ExtensionRepository.java
@@ -26,6 +26,8 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Future;
 
 public interface ExtensionRepository {
+    void initialize() throws IOException;
+
     BundleAvailability getBundleAvailability(BundleCoordinate bundleCoordinate) throws IOException;
 
     Future<Set<Bundle>> fetch(Set<BundleCoordinate> bundleCoordinates, ExecutorService executorService, int concurrentDownloads);
diff --git a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/extensions/FileSystemExtensionRepository.java b/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/extensions/FileSystemExtensionRepository.java
index 173d7d9..d007633 100644
--- a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/extensions/FileSystemExtensionRepository.java
+++ b/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/extensions/FileSystemExtensionRepository.java
@@ -31,34 +31,58 @@ import org.slf4j.LoggerFactory;
 import java.io.File;
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
 import java.util.Collections;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Future;
-import java.util.function.Function;
 
 public class FileSystemExtensionRepository implements ExtensionRepository {
     private static final Logger logger = LoggerFactory.getLogger(FileSystemExtensionRepository.class);
 
     private final ExtensionDiscoveringManager extensionManager;
     private final NarClassLoaders narClassLoaders;
-    private final File narLibDirectory;
+    private final File writableLibDirectory;
+    private final Set<File> readOnlyExtensionDirectories;
     private final File workingDirectory;
     private final List<ExtensionClient> clients;
 
 
-    public FileSystemExtensionRepository(final ExtensionDiscoveringManager extensionManager, final File narLibDirectory, final File workingDirectory,
-                                         final NarClassLoaders narClassLoaders, final List<ExtensionClient> clients) {
+    public FileSystemExtensionRepository(final ExtensionDiscoveringManager extensionManager, final File writableLibDirectory, final Collection<File> readOnlyExtensionDirectories,
+                                         final File workingDirectory, final NarClassLoaders narClassLoaders, final List<ExtensionClient> clients) {
         this.extensionManager = extensionManager;
-        this.narLibDirectory = narLibDirectory;
+        this.writableLibDirectory = writableLibDirectory;
+        this.readOnlyExtensionDirectories = readOnlyExtensionDirectories == null ? Collections.emptySet() : new HashSet<>(readOnlyExtensionDirectories);
         this.workingDirectory = workingDirectory;
         this.narClassLoaders = narClassLoaders;
         this.clients = clients;
     }
 
     @Override
+    public void initialize() throws IOException {
+        if (readOnlyExtensionDirectories.isEmpty()) {
+            return;
+        }
+
+        final Set<File> readOnlyNars = new HashSet<>();
+        for (final File extensionDir : readOnlyExtensionDirectories) {
+            final File[] narFiles = extensionDir.listFiles(file -> file.getName().endsWith(".nar"));
+            if (narFiles == null) {
+                logger.warn("Failed to perform listing of read-only extensions directory {}. Will not load extensions from this directory.", extensionDir.getAbsolutePath());
+                continue;
+            }
+
+            readOnlyNars.addAll(Arrays.asList(narFiles));
+        }
+
+        loadExtensions(readOnlyNars);
+    }
+
+    @Override
     public BundleAvailability getBundleAvailability(final BundleCoordinate bundleCoordinate) {
         final Bundle bundle = extensionManager.getBundle(bundleCoordinate);
         if (bundle == null) {
@@ -87,29 +111,29 @@ public class FileSystemExtensionRepository implements ExtensionRepository {
             return CompletableFuture.completedFuture(Collections.emptySet());
         }
 
-        final DownloadQueue downloadQueue = new DownloadQueue(extensionManager, executorService, concurrentDownloads, bundleCoordinates, narLibDirectory, clients);
+        final DownloadQueue downloadQueue = new DownloadQueue(extensionManager, executorService, concurrentDownloads, bundleCoordinates, writableLibDirectory, clients);
         final CompletableFuture<Void> downloadFuture = downloadQueue.download();
         logger.info("Beginning download of extensions {}", bundleCoordinates);
 
-        final CompletableFuture<Set<Bundle>> loadFuture = downloadFuture.thenApply(new Function<Void, Set<Bundle>>() {
-            @Override
-            public Set<Bundle> apply(final Void aVoid) {
-                final Set<File> downloadedFiles = downloadQueue.getDownloadedFiles();
-
-                try {
-                    return loadExtensions(downloadedFiles);
-                } catch (final Exception e) {
-                    throw new RuntimeException("Could not load extensions", e);
-                }
-            }
-        });
+        // When the download completes, load the extensions & return that future.
+        final CompletableFuture<Set<Bundle>> loadFuture = downloadFuture.thenApply(voidDownloadResult -> loadExtensions(downloadQueue));
 
         return loadFuture;
     }
 
+    private Set<Bundle> loadExtensions(final DownloadQueue downloadQueue) {
+        final Set<File> downloadedFiles = downloadQueue.getDownloadedFiles();
+        logger.info("Completed download of {} bundles. Unpacking NAR files now", downloadedFiles.size());
+
+        try {
+            return loadExtensions(downloadedFiles);
+        } catch (final Exception e) {
+            throw new RuntimeException("Could not load extensions", e);
+        }
+    }
+
     private Set<Bundle> loadExtensions(final Set<File> downloadedFiles) throws IOException {
         final List<File> unpackedDirs = new ArrayList<>();
-        logger.info("Completed download of {} bundles. Unpacking NAR files now", downloadedFiles.size());
 
         final long start = System.currentTimeMillis();
         for (final File downloadedFile : downloadedFiles) {
diff --git a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/extensions/ExtensionRepository.java b/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/engine/FlowPurgeAction.java
similarity index 60%
copy from nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/extensions/ExtensionRepository.java
copy to nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/engine/FlowPurgeAction.java
index 291a301..6404b4f 100644
--- a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/extensions/ExtensionRepository.java
+++ b/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/engine/FlowPurgeAction.java
@@ -15,18 +15,8 @@
  * limitations under the License.
  */
 
-package org.apache.nifi.extensions;
+package org.apache.nifi.stateless.engine;
 
-import org.apache.nifi.bundle.Bundle;
-import org.apache.nifi.bundle.BundleCoordinate;
-
-import java.io.IOException;
-import java.util.Set;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Future;
-
-public interface ExtensionRepository {
-    BundleAvailability getBundleAvailability(BundleCoordinate bundleCoordinate) throws IOException;
-
-    Future<Set<Bundle>> fetch(Set<BundleCoordinate> bundleCoordinates, ExecutorService executorService, int concurrentDownloads);
+public interface FlowPurgeAction {
+    void purge();
 }
diff --git a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/engine/StandardExecutionProgress.java b/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/engine/StandardExecutionProgress.java
index 12427fb..4502be2 100644
--- a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/engine/StandardExecutionProgress.java
+++ b/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/engine/StandardExecutionProgress.java
@@ -62,6 +62,7 @@ public class StandardExecutionProgress implements ExecutionProgress {
     private final StatelessStateManagerProvider stateManagerProvider;
     private final Long maxProvenanceEventId;
     private final DataflowTriggerContext triggerContext;
+    private final FlowPurgeAction purgeAction;
 
     private final BlockingQueue<CompletionAction> completionActionQueue;
     private volatile boolean canceled = false;
@@ -69,7 +70,7 @@ public class StandardExecutionProgress implements ExecutionProgress {
 
     public StandardExecutionProgress(final ProcessGroup rootGroup, final List<FlowFileQueue> internalFlowFileQueues, final BlockingQueue<TriggerResult> resultQueue,
                                      final RepositoryContextFactory repositoryContextFactory, final Set<String> failurePortNames, final AsynchronousCommitTracker commitTracker,
-                                     final StatelessStateManagerProvider stateManagerProvider, final DataflowTriggerContext triggerContext) {
+                                     final StatelessStateManagerProvider stateManagerProvider, final DataflowTriggerContext triggerContext, final FlowPurgeAction purgeAction) {
         this.rootGroup = rootGroup;
         this.internalFlowFileQueues = internalFlowFileQueues;
         this.resultQueue = resultQueue;
@@ -80,6 +81,7 @@ public class StandardExecutionProgress implements ExecutionProgress {
         this.stateManagerProvider = stateManagerProvider;
         this.maxProvenanceEventId = provenanceRepository.getMaxEventId();
         this.triggerContext = triggerContext;
+        this.purgeAction = purgeAction;
 
         completionActionQueue = new LinkedBlockingQueue<>();
     }
@@ -243,8 +245,8 @@ public class StandardExecutionProgress implements ExecutionProgress {
         commitTracker.triggerFailureCallbacks(new RuntimeException("Dataflow Canceled"));
         stateManagerProvider.rollbackUpdates();
         completionActionQueue.offer(CompletionAction.CANCEL);
+        purgeAction.purge();
         resultQueue.offer(new CanceledTriggerResult());
-        contentRepository.purge();
     }
 
     @Override
@@ -252,8 +254,8 @@ public class StandardExecutionProgress implements ExecutionProgress {
         commitTracker.triggerFailureCallbacks(cause);
         stateManagerProvider.rollbackUpdates();
         completionActionQueue.offer(CompletionAction.CANCEL);
+        purgeAction.purge();
         resultQueue.offer(new ExceptionalTriggerResult(cause));
-        contentRepository.purge();
     }
 
     public Map<String, List<FlowFile>> drainOutputQueues() {
diff --git a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/engine/StandardStatelessEngine.java b/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/engine/StandardStatelessEngine.java
index 16488f4..24c1ddd 100644
--- a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/engine/StandardStatelessEngine.java
+++ b/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/engine/StandardStatelessEngine.java
@@ -96,7 +96,7 @@ import static java.util.Objects.requireNonNull;
 
 public class StandardStatelessEngine implements StatelessEngine<VersionedFlowSnapshot> {
     private static final Logger logger = LoggerFactory.getLogger(StandardStatelessEngine.class);
-    private static final int CONCURRENT_EXTENSION_DOWNLOADS = 4;
+    private static final int CONCURRENT_EXTENSION_DOWNLOADS = 8;
 
     // Member Variables injected via Builder
     private final ExtensionManager extensionManager;
@@ -123,7 +123,6 @@ public class StandardStatelessEngine implements StatelessEngine<VersionedFlowSna
     private RepositoryContextFactory repositoryContextFactory;
     private boolean initialized = false;
 
-
     private StandardStatelessEngine(final Builder builder) {
         this.extensionManager = requireNonNull(builder.extensionManager, "Extension Manager must be provided");
         this.bulletinRepository = requireNonNull(builder.bulletinRepository, "Bulletin Repository must be provided");
@@ -207,7 +206,7 @@ public class StandardStatelessEngine implements StatelessEngine<VersionedFlowSna
         // Create a Composite Parameter Value Provider that wraps all of the others.
         final CompositeParameterValueProvider provider = new CompositeParameterValueProvider(providers);
         final ParameterValueProviderInitializationContext initializationContext =
-                new StandardParameterValueProviderInitializationContext(provider, Collections.emptyMap(), UUID.randomUUID().toString());
+            new StandardParameterValueProviderInitializationContext(provider, Collections.emptyMap(), UUID.randomUUID().toString());
         provider.initialize(initializationContext);
         return provider;
     }
@@ -292,7 +291,6 @@ public class StandardStatelessEngine implements StatelessEngine<VersionedFlowSna
         }
     }
 
-
     private void loadNecessaryExtensions(final DataflowDefinition<VersionedFlowSnapshot> dataflowDefinition) {
         final VersionedProcessGroup group = dataflowDefinition.getFlowSnapshot().getFlowContents();
         final Set<BundleCoordinate> requiredBundles = gatherRequiredBundles(group);
@@ -315,11 +313,12 @@ public class StandardStatelessEngine implements StatelessEngine<VersionedFlowSna
             requiredBundles.add(coordinate);
         }
 
+        final Set<BundleCoordinate> unavailableBundles = determineUnavailableBundles(requiredBundles);
         final ExecutorService executor = new FlowEngine(CONCURRENT_EXTENSION_DOWNLOADS, "Download Extensions", true);
-        final Future<Set<Bundle>> future = extensionRepository.fetch(requiredBundles, executor, CONCURRENT_EXTENSION_DOWNLOADS);
+        final Future<Set<Bundle>> future = extensionRepository.fetch(unavailableBundles, executor, CONCURRENT_EXTENSION_DOWNLOADS);
         executor.shutdown();
 
-        logger.info("Waiting for bundles to complete download...");
+        logger.info("Waiting for {} bundles to complete download...", unavailableBundles.size());
         final long downloadStart = System.currentTimeMillis();
         final Set<Bundle> downloadedBundles;
         try {
@@ -333,6 +332,26 @@ public class StandardStatelessEngine implements StatelessEngine<VersionedFlowSna
         logger.info("Successfully downloaded {} bundles in {} millis", downloadedBundles.size(), downloadMillis);
     }
 
+    private Set<BundleCoordinate> determineUnavailableBundles(final Set<BundleCoordinate> coordinates) {
+        final Set<BundleCoordinate> unavailable = new HashSet<>();
+        determineUnavailableBundles(coordinates, unavailable);
+        return unavailable;
+    }
+
+    private void determineUnavailableBundles(final Set<BundleCoordinate> coordinates, final Set<BundleCoordinate> unavailable) {
+        for (final BundleCoordinate coordinate : coordinates) {
+            final Bundle bundle = extensionManager.getBundle(coordinate);
+            if (bundle == null) {
+                unavailable.add(coordinate);
+            } else {
+                final BundleCoordinate parentCoordinate = bundle.getBundleDetails().getDependencyCoordinate();
+                if (parentCoordinate != null) {
+                    determineUnavailableBundles(Collections.singleton(parentCoordinate), unavailable);
+                }
+            }
+        }
+    }
+
     private Set<BundleCoordinate> gatherRequiredBundles(final VersionedProcessGroup group) {
         final Set<BundleCoordinate> requiredBundles = new HashSet<>();
         gatherRequiredBundles(group, requiredBundles);
@@ -420,7 +439,6 @@ public class StandardStatelessEngine implements StatelessEngine<VersionedFlowSna
         return resolved;
     }
 
-
     private BundleCoordinate determineBundleCoordinate(final ConfigurableExtensionDefinition extensionDefinition, final String extensionType) {
         final String explicitCoordinates = extensionDefinition.getBundleCoordinates();
         if (explicitCoordinates != null && !explicitCoordinates.trim().isEmpty()) {
@@ -475,7 +493,6 @@ public class StandardStatelessEngine implements StatelessEngine<VersionedFlowSna
         return new BundleCoordinate(splits[0], splits[1], splits[2]);
     }
 
-
     private String resolveExtensionClassName(final ConfigurableExtensionDefinition extensionDefinition, final String extensionType) {
         final String specifiedType = extensionDefinition.getType();
         if (specifiedType.contains(".")) {
@@ -527,7 +544,6 @@ public class StandardStatelessEngine implements StatelessEngine<VersionedFlowSna
         }
     }
 
-
     private void registerParameterContext(final ParameterContextDefinition parameterContextDefinition, final Map<String, ParameterContext> parameterContextMap) {
         final String contextName = parameterContextDefinition.getName();
         final ParameterContext existingContext = parameterContextMap.get(contextName);
diff --git a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/flow/StandardDataflowDefinition.java b/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/flow/StandardDataflowDefinition.java
index b3e090e..e1bb1e6 100644
--- a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/flow/StandardDataflowDefinition.java
+++ b/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/flow/StandardDataflowDefinition.java
@@ -19,9 +19,10 @@ package org.apache.nifi.stateless.flow;
 
 import org.apache.nifi.flow.Bundle;
 import org.apache.nifi.flow.VersionedControllerService;
-import org.apache.nifi.registry.flow.VersionedFlowSnapshot;
+import org.apache.nifi.flow.VersionedPort;
 import org.apache.nifi.flow.VersionedProcessGroup;
 import org.apache.nifi.flow.VersionedProcessor;
+import org.apache.nifi.registry.flow.VersionedFlowSnapshot;
 import org.apache.nifi.stateless.config.ParameterContextDefinition;
 import org.apache.nifi.stateless.config.ParameterValueProviderDefinition;
 import org.apache.nifi.stateless.config.ReportingTaskDefinition;
@@ -30,6 +31,7 @@ import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
+import java.util.stream.Collectors;
 
 import static java.util.Objects.requireNonNull;
 
@@ -68,6 +70,20 @@ public class StandardDataflowDefinition implements DataflowDefinition<VersionedF
     }
 
     @Override
+    public Set<String> getInputPortNames() {
+        return flowSnapshot.getFlowContents().getInputPorts().stream()
+            .map(VersionedPort::getName)
+            .collect(Collectors.toSet());
+    }
+
+    @Override
+    public Set<String> getOutputPortNames() {
+        return flowSnapshot.getFlowContents().getOutputPorts().stream()
+            .map(VersionedPort::getName)
+            .collect(Collectors.toSet());
+    }
+
+    @Override
     public List<ParameterContextDefinition> getParameterContexts() {
         return parameterContexts;
     }
diff --git a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/flow/StandardStatelessDataflowFactory.java b/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/flow/StandardStatelessDataflowFactory.java
index b15cb58..3f64964 100644
--- a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/flow/StandardStatelessDataflowFactory.java
+++ b/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/flow/StandardStatelessDataflowFactory.java
@@ -142,8 +142,9 @@ public class StandardStatelessDataflowFactory implements StatelessDataflowFactor
                 extensionClients.add(extensionClient);
             }
 
-            final ExtensionRepository extensionRepository = new FileSystemExtensionRepository(extensionManager, engineConfiguration.getExtensionsDirectory(), engineConfiguration.getWorkingDirectory(),
-                narClassLoaders, extensionClients);
+            final ExtensionRepository extensionRepository = new FileSystemExtensionRepository(extensionManager, engineConfiguration.getExtensionsDirectory(),
+                engineConfiguration.getReadOnlyExtensionsDirectories(), engineConfiguration.getWorkingDirectory(), narClassLoaders, extensionClients);
+            extensionRepository.initialize();
 
             final VariableRegistry variableRegistry = VariableRegistry.EMPTY_REGISTRY;
             final PropertyEncryptor lazyInitializedEncryptor = new PropertyEncryptor() {
diff --git a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/flow/StandardStatelessFlow.java b/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/flow/StandardStatelessFlow.java
index 344d747..a64aa57 100644
--- a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/flow/StandardStatelessFlow.java
+++ b/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/flow/StandardStatelessFlow.java
@@ -433,7 +433,7 @@ public class StandardStatelessFlow implements StatelessDataflow {
         final BlockingQueue<TriggerResult> resultQueue = new LinkedBlockingQueue<>();
 
         final ExecutionProgress executionProgress = new StandardExecutionProgress(rootGroup, internalFlowFileQueues, resultQueue,
-            repositoryContextFactory, dataflowDefinition.getFailurePortNames(), tracker, stateManagerProvider, triggerContext);
+            repositoryContextFactory, dataflowDefinition.getFailurePortNames(), tracker, stateManagerProvider, triggerContext, this::purge);
 
         final AtomicReference<Future<?>> processFuture = new AtomicReference<>();
         final DataflowTrigger trigger = new DataflowTrigger() {
@@ -598,6 +598,8 @@ public class StandardStatelessFlow implements StatelessDataflow {
             ((DrainableFlowFileQueue) connection.getFlowFileQueue()).drainTo(flowFiles);
             flowFiles.clear();
         }
+
+        repositoryContextFactory.getContentRepository().purge();
     }
 
     @Override
diff --git a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/test/java/org/apache/nifi/stateless/config/TestPropertiesFileFlowDefinitionParser.java b/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/test/java/org/apache/nifi/stateless/config/TestPropertiesFileFlowDefinitionParser.java
index 695c253..4619f58 100644
--- a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/test/java/org/apache/nifi/stateless/config/TestPropertiesFileFlowDefinitionParser.java
+++ b/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/test/java/org/apache/nifi/stateless/config/TestPropertiesFileFlowDefinitionParser.java
@@ -27,6 +27,7 @@ import java.io.File;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
@@ -94,6 +95,11 @@ public class TestPropertiesFileFlowDefinitionParser {
             }
 
             @Override
+            public Collection<File> getReadOnlyExtensionsDirectories() {
+                return Collections.emptyList();
+            }
+
+            @Override
             public File getKrb5File() {
                 return null;
             }
diff --git a/nifi-system-tests/nifi-stateless-system-test-suite/src/test/java/org/apache/nifi/stateless/StatelessSystemIT.java b/nifi-system-tests/nifi-stateless-system-test-suite/src/test/java/org/apache/nifi/stateless/StatelessSystemIT.java
index 603d051..328bf37 100644
--- a/nifi-system-tests/nifi-stateless-system-test-suite/src/test/java/org/apache/nifi/stateless/StatelessSystemIT.java
+++ b/nifi-system-tests/nifi-stateless-system-test-suite/src/test/java/org/apache/nifi/stateless/StatelessSystemIT.java
@@ -19,6 +19,7 @@ package org.apache.nifi.stateless;
 
 import com.fasterxml.jackson.databind.ObjectMapper;
 import org.apache.nifi.flow.Bundle;
+import org.apache.nifi.flow.VersionedPort;
 import org.apache.nifi.registry.flow.VersionedFlowSnapshot;
 import org.apache.nifi.stateless.bootstrap.StatelessBootstrap;
 import org.apache.nifi.stateless.config.ExtensionClientDefinition;
@@ -42,11 +43,13 @@ import java.io.FileInputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
 import java.util.Optional;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
 
 public class StatelessSystemIT {
     private final List<StatelessDataflow> createdFlows = new ArrayList<>();
@@ -90,6 +93,11 @@ public class StatelessSystemIT {
             }
 
             @Override
+            public Collection<File> getReadOnlyExtensionsDirectories() {
+                return Collections.emptyList();
+            }
+
+            @Override
             public File getKrb5File() {
                 return new File("/etc/krb5.conf");
             }
@@ -171,6 +179,20 @@ public class StatelessSystemIT {
             }
 
             @Override
+            public Set<String> getInputPortNames() {
+                return versionedFlowSnapshot.getFlowContents().getInputPorts().stream()
+                    .map(VersionedPort::getName)
+                    .collect(Collectors.toSet());
+            }
+
+            @Override
+            public Set<String> getOutputPortNames() {
+                return versionedFlowSnapshot.getFlowContents().getOutputPorts().stream()
+                    .map(VersionedPort::getName)
+                    .collect(Collectors.toSet());
+            }
+
+            @Override
             public List<ParameterContextDefinition> getParameterContexts() {
                 return parameterContexts;
             }