You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by ma...@apache.org on 2022/06/24 17:13:48 UTC

[nifi] branch main updated: NIFI-10111: Added option to unpack NARs into an Uber Jar and updated stateless nifi to always use this.

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

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


The following commit(s) were added to refs/heads/main by this push:
     new 1442dcef23 NIFI-10111: Added option to unpack NARs into an Uber Jar and updated stateless nifi to always use this.
1442dcef23 is described below

commit 1442dcef233d627b1e34fee15c0d012005f749f2
Author: Mark Payne <ma...@hotmail.com>
AuthorDate: Wed May 18 09:46:50 2022 -0400

    NIFI-10111: Added option to unpack NARs into an Uber Jar and updated stateless nifi to always use this.
    
    Signed-off-by: Matthew Burgess <ma...@apache.org>
    
    This closes #6117
---
 .../main/java/org/apache/nifi/minifi/MiNiFi.java   |   4 +-
 .../java/org/apache/nifi/util/NiFiProperties.java  |  10 +
 .../src/main/asciidoc/administration-guide.adoc    |   3 +
 .../nifi/documentation/DocGeneratorTest.java       |   3 +-
 .../org/apache/nifi/nar/StandardNarLoader.java     |   7 +-
 .../org/apache/nifi/nar/AbstractTestNarLoader.java |   5 +-
 .../java/org/apache/nifi/nar/NarUnpackerTest.java  |   8 +-
 .../apache/nifi/headless/HeadlessNiFiServer.java   |   5 +-
 .../java/org/apache/nifi/nar/FileDigestUtils.java  |   2 +-
 .../java/org/apache/nifi/nar/NarUnpackMode.java    |  37 ++++
 .../main/java/org/apache/nifi/nar/NarUnpacker.java | 241 ++++++++++++++++++---
 .../nifi-framework/nifi-resources/pom.xml          |   1 +
 .../src/main/resources/conf/nifi.properties        |   1 +
 .../src/main/java/org/apache/nifi/NiFi.java        |   4 +-
 .../main/java/org/apache/nifi/StatelessNiFi.java   |   3 +-
 .../org/apache/nifi/web/server/JettyServer.java    |   5 +-
 .../accesscontrol/AccessControlHelper.java         |  17 +-
 .../OneWaySslAccessControlHelper.java              |  13 +-
 .../stateless/bootstrap/StatelessBootstrap.java    |   6 +-
 .../extensions/FileSystemExtensionRepository.java  |   3 +-
 20 files changed, 323 insertions(+), 55 deletions(-)

diff --git a/minifi/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-runtime/src/main/java/org/apache/nifi/minifi/MiNiFi.java b/minifi/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-runtime/src/main/java/org/apache/nifi/minifi/MiNiFi.java
index c83d008b72..2849838fbc 100644
--- a/minifi/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-runtime/src/main/java/org/apache/nifi/minifi/MiNiFi.java
+++ b/minifi/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-runtime/src/main/java/org/apache/nifi/minifi/MiNiFi.java
@@ -24,6 +24,7 @@ import org.apache.nifi.nar.NarClassLoaders;
 import org.apache.nifi.nar.NarClassLoadersHolder;
 import org.apache.nifi.nar.NarUnpacker;
 import org.apache.nifi.nar.SystemBundle;
+import org.apache.nifi.nar.NarUnpackMode;
 import org.apache.nifi.util.FileUtils;
 import org.apache.nifi.util.NiFiProperties;
 import org.slf4j.Logger;
@@ -124,7 +125,8 @@ public class MiNiFi {
         final Bundle systemBundle = SystemBundle.create(properties, rootClassLoader);
 
         // expand the nars
-        final ExtensionMapping extensionMapping = NarUnpacker.unpackNars(properties, FRAMEWORK_NAR_ID, systemBundle);
+        final NarUnpackMode unpackMode = properties.isUnpackNarsToUberJar() ? NarUnpackMode.UNPACK_TO_UBER_JAR : NarUnpackMode.UNPACK_INDIVIDUAL_JARS;
+        final ExtensionMapping extensionMapping = NarUnpacker.unpackNars(properties, FRAMEWORK_NAR_ID, systemBundle, unpackMode);
 
         // load the extensions classloaders
         NarClassLoaders narClassLoaders = NarClassLoadersHolder.getInstance();
diff --git a/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java b/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java
index 995c720271..f00a29419d 100644
--- a/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java
+++ b/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java
@@ -72,6 +72,7 @@ public class NiFiProperties extends ApplicationProperties {
     public static final String NAR_LIBRARY_DIRECTORY_PREFIX = "nifi.nar.library.directory.";
     public static final String NAR_LIBRARY_AUTOLOAD_DIRECTORY = "nifi.nar.library.autoload.directory";
     public static final String NAR_WORKING_DIRECTORY = "nifi.nar.working.directory";
+    public static final String UNPACK_NARS_TO_UBER_JAR = "nifi.nar.unpack.uber.jar";
     public static final String COMPONENT_DOCS_DIRECTORY = "nifi.documentation.working.directory";
     public static final String SENSITIVE_PROPS_KEY = "nifi.sensitive.props.key";
     public static final String SENSITIVE_PROPS_ALGORITHM = "nifi.sensitive.props.algorithm";
@@ -344,6 +345,7 @@ public class NiFiProperties extends ApplicationProperties {
     public static final int DEFAULT_WEB_MAX_ACCESS_TOKEN_REQUESTS_PER_SECOND = 25;
     public static final String DEFAULT_WEB_REQUEST_TIMEOUT = "60 secs";
     public static final String DEFAULT_NAR_WORKING_DIR = "./work/nar";
+    public static final boolean DEFAULT_UNPACK_NARS_TO_UBER_JAR = false;
     public static final String DEFAULT_COMPONENT_DOCS_DIRECTORY = "./work/docs/components";
     public static final String DEFAULT_NAR_LIBRARY_DIR = "./lib";
     public static final String DEFAULT_NAR_LIBRARY_AUTOLOAD_DIR = "./extensions";
@@ -768,6 +770,14 @@ public class NiFiProperties extends ApplicationProperties {
         return new File(getProperty(NAR_WORKING_DIRECTORY, DEFAULT_NAR_WORKING_DIR));
     }
 
+    public boolean isUnpackNarsToUberJar() {
+        final String propertyValue = getProperty(UNPACK_NARS_TO_UBER_JAR);
+        if (propertyValue == null) {
+            return DEFAULT_UNPACK_NARS_TO_UBER_JAR;
+        }
+        return Boolean.parseBoolean(propertyValue);
+    }
+
     public File getFrameworkWorkingDirectory() {
         return new File(getNarWorkingDirectory(), "framework");
     }
diff --git a/nifi-docs/src/main/asciidoc/administration-guide.adoc b/nifi-docs/src/main/asciidoc/administration-guide.adoc
index 11df488013..f27d96da19 100644
--- a/nifi-docs/src/main/asciidoc/administration-guide.adoc
+++ b/nifi-docs/src/main/asciidoc/administration-guide.adoc
@@ -3075,6 +3075,9 @@ For example, to provide two additional library locations, a user could also spec
 Providing three total locations, including  `nifi.nar.library.directory`.
 |`nifi.nar.working.directory`|The location of the nar working directory. The default value is `./work/nar` and probably should be left as is.
 |`nifi.documentation.working.directory`|The documentation working directory. The default value is `./work/docs/components` and probably should be left as is.
+|`nifi.nar.unpack.uber.jar`|If set to `true`, when a nar file is unpacked, the inner jar files will be unpacked into a single jar file instead of individual jar files. This can result in NiFi taking
+longer to startup for the first time (about 1-2 minutes, typically) but can result in far fewer open file handles, which can be helpful in certain environments. The default value is `false`. This
+feature is considered experimental. Changing the value of this property may not take effect unless the working directory is also deleted.
 |`nifi.processor.scheduling.timeout`|Time to wait for a Processor's life-cycle operation (`@OnScheduled` and `@OnUnscheduled`) to finish before other life-cycle operation (e.g., *stop*) could be invoked. The default value is `1 min`.
 |===
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/DocGeneratorTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/DocGeneratorTest.java
index 3e93427e66..749d938820 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/DocGeneratorTest.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/DocGeneratorTest.java
@@ -25,6 +25,7 @@ import org.apache.nifi.nar.NarClassLoadersHolder;
 import org.apache.nifi.nar.NarUnpacker;
 import org.apache.nifi.nar.StandardExtensionDiscoveringManager;
 import org.apache.nifi.nar.SystemBundle;
+import org.apache.nifi.nar.NarUnpackMode;
 import org.apache.nifi.util.NiFiProperties;
 import org.junit.Assert;
 import org.junit.Test;
@@ -51,7 +52,7 @@ public class DocGeneratorTest {
                 temporaryFolder.getRoot().getAbsolutePath());
 
         final Bundle systemBundle = SystemBundle.create(properties);
-        final ExtensionMapping mapping = NarUnpacker.unpackNars(properties, systemBundle);
+        final ExtensionMapping mapping = NarUnpacker.unpackNars(properties, systemBundle, NarUnpackMode.UNPACK_INDIVIDUAL_JARS);
 
         NarClassLoadersHolder.getInstance().init(properties.getFrameworkWorkingDirectory(), properties.getExtensionsWorkingDirectory());
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/main/java/org/apache/nifi/nar/StandardNarLoader.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/main/java/org/apache/nifi/nar/StandardNarLoader.java
index b027dcaf49..4920e90dc6 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/main/java/org/apache/nifi/nar/StandardNarLoader.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/main/java/org/apache/nifi/nar/StandardNarLoader.java
@@ -51,6 +51,7 @@ public class StandardNarLoader implements NarLoader {
     private final ExtensionDiscoveringManager extensionManager;
     private final ExtensionMapping extensionMapping;
     private final ExtensionUiLoader extensionUiLoader;
+    private final NarUnpackMode narUnpackMode;
 
     private Set<BundleDetails> previouslySkippedBundles;
 
@@ -59,13 +60,15 @@ public class StandardNarLoader implements NarLoader {
                              final NarClassLoaders narClassLoaders,
                              final ExtensionDiscoveringManager extensionManager,
                              final ExtensionMapping extensionMapping,
-                             final ExtensionUiLoader extensionUiLoader) {
+                             final ExtensionUiLoader extensionUiLoader,
+                             final NarUnpackMode narUnpackMode) {
         this.extensionsWorkingDir = extensionsWorkingDir;
         this.docsWorkingDir = docsWorkingDir;
         this.narClassLoaders = narClassLoaders;
         this.extensionManager = extensionManager;
         this.extensionMapping = extensionMapping;
         this.extensionUiLoader = extensionUiLoader;
+        this.narUnpackMode = narUnpackMode;
     }
 
     @Override
@@ -161,7 +164,7 @@ public class StandardNarLoader implements NarLoader {
                 return null;
             }
 
-            final File unpackedExtension = NarUnpacker.unpackNar(narFile, extensionsWorkingDir, true);
+            final File unpackedExtension = NarUnpacker.unpackNar(narFile, extensionsWorkingDir, true, narUnpackMode);
             NarUnpacker.mapExtension(unpackedExtension, coordinate, docsWorkingDir, extensionMapping);
             return unpackedExtension;
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/test/java/org/apache/nifi/nar/AbstractTestNarLoader.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/test/java/org/apache/nifi/nar/AbstractTestNarLoader.java
index 481eae7003..a34ef92c7c 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/test/java/org/apache/nifi/nar/AbstractTestNarLoader.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/test/java/org/apache/nifi/nar/AbstractTestNarLoader.java
@@ -71,7 +71,7 @@ public abstract class AbstractTestNarLoader {
 
         // Unpack NARs
         systemBundle = SystemBundle.create(properties);
-        extensionMapping = NarUnpacker.unpackNars(properties, systemBundle);
+        extensionMapping = NarUnpacker.unpackNars(properties, systemBundle, NarUnpackMode.UNPACK_INDIVIDUAL_JARS);
         assertEquals(0, extensionMapping.getAllExtensionNames().size());
 
         // Initialize NarClassLoaders
@@ -96,7 +96,8 @@ public abstract class AbstractTestNarLoader {
                 extensionManager,
                 extensionMapping,
                 (bundles) -> {
-                });
+                },
+                NarUnpackMode.UNPACK_INDIVIDUAL_JARS);
     }
 
     private void deleteDir(String path) throws IOException {
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/test/java/org/apache/nifi/nar/NarUnpackerTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/test/java/org/apache/nifi/nar/NarUnpackerTest.java
index cc7fe74c62..e54f9dc773 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/test/java/org/apache/nifi/nar/NarUnpackerTest.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/test/java/org/apache/nifi/nar/NarUnpackerTest.java
@@ -90,7 +90,7 @@ public class NarUnpackerTest {
         assertEquals("./target/NarUnpacker/lib2/",
                 properties.getProperty("nifi.nar.library.directory.alt"));
 
-        final ExtensionMapping extensionMapping = NarUnpacker.unpackNars(properties, SystemBundle.create(properties));
+        final ExtensionMapping extensionMapping = NarUnpacker.unpackNars(properties, SystemBundle.create(properties), NarUnpackMode.UNPACK_INDIVIDUAL_JARS);
 
         assertEquals(2, extensionMapping.getAllExtensionNames().size());
 
@@ -122,7 +122,7 @@ public class NarUnpackerTest {
         others.put("nifi.nar.library.directory.alt", emptyDir.toString());
         NiFiProperties properties = loadSpecifiedProperties("/NarUnpacker/conf/nifi.properties", others);
 
-        final ExtensionMapping extensionMapping = NarUnpacker.unpackNars(properties, SystemBundle.create(properties));
+        final ExtensionMapping extensionMapping = NarUnpacker.unpackNars(properties, SystemBundle.create(properties), NarUnpackMode.UNPACK_INDIVIDUAL_JARS);
 
         assertEquals(1, extensionMapping.getAllExtensionNames().size());
         assertTrue(extensionMapping.getAllExtensionNames().keySet().contains("org.apache.nifi.processors.dummy.one"));
@@ -149,7 +149,7 @@ public class NarUnpackerTest {
         others.put("nifi.nar.library.directory.alt", nonExistantDir.toString());
         NiFiProperties properties = loadSpecifiedProperties("/NarUnpacker/conf/nifi.properties", others);
 
-        final ExtensionMapping extensionMapping = NarUnpacker.unpackNars(properties, SystemBundle.create(properties));
+        final ExtensionMapping extensionMapping = NarUnpacker.unpackNars(properties, SystemBundle.create(properties), NarUnpackMode.UNPACK_INDIVIDUAL_JARS);
 
         assertTrue(extensionMapping.getAllExtensionNames().keySet().contains("org.apache.nifi.processors.dummy.one"));
 
@@ -177,7 +177,7 @@ public class NarUnpackerTest {
         others.put("nifi.nar.library.directory.alt", nonDir.toString());
         NiFiProperties properties = loadSpecifiedProperties("/NarUnpacker/conf/nifi.properties", others);
 
-        final ExtensionMapping extensionMapping = NarUnpacker.unpackNars(properties, SystemBundle.create(properties));
+        final ExtensionMapping extensionMapping = NarUnpacker.unpackNars(properties, SystemBundle.create(properties), NarUnpackMode.UNPACK_INDIVIDUAL_JARS);
 
         assertNull(extensionMapping);
     }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-headless-server/src/main/java/org/apache/nifi/headless/HeadlessNiFiServer.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-headless-server/src/main/java/org/apache/nifi/headless/HeadlessNiFiServer.java
index 0d726eceb4..17cdfd0060 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-headless-server/src/main/java/org/apache/nifi/headless/HeadlessNiFiServer.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-headless-server/src/main/java/org/apache/nifi/headless/HeadlessNiFiServer.java
@@ -54,6 +54,7 @@ import org.apache.nifi.nar.NarClassLoadersHolder;
 import org.apache.nifi.nar.NarLoader;
 import org.apache.nifi.nar.StandardExtensionDiscoveringManager;
 import org.apache.nifi.nar.StandardNarLoader;
+import org.apache.nifi.nar.NarUnpackMode;
 import org.apache.nifi.registry.VariableRegistry;
 import org.apache.nifi.registry.flow.StandardFlowRegistryClient;
 import org.apache.nifi.registry.variable.FileBasedVariableRegistry;
@@ -170,13 +171,15 @@ public class HeadlessNiFiServer implements NiFiServer {
             FlowManager flowManager = flowController.getFlowManager();
             flowManager.getGroup(flowManager.getRootGroupId()).startProcessing();
 
+            final NarUnpackMode unpackMode = props.isUnpackNarsToUberJar() ? NarUnpackMode.UNPACK_TO_UBER_JAR : NarUnpackMode.UNPACK_INDIVIDUAL_JARS;
             final NarLoader narLoader = new StandardNarLoader(
                     props.getExtensionsWorkingDirectory(),
                     props.getComponentDocumentationWorkingDirectory(),
                     NarClassLoadersHolder.getInstance(),
                     extensionManager,
                     new ExtensionMapping(), // Mapping is for documentation which is for the UI, not headless
-                    null); // UI Loader is for documentation which is for the UI, not headless
+                    null,
+                    unpackMode); // UI Loader is for documentation which is for the UI, not headless
 
             narAutoLoader = new NarAutoLoader(props, narLoader);
             narAutoLoader.start();
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/FileDigestUtils.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/FileDigestUtils.java
index ab8495865f..2bb8152816 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/FileDigestUtils.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/FileDigestUtils.java
@@ -29,7 +29,7 @@ import java.security.NoSuchAlgorithmException;
 public final class FileDigestUtils {
     private static final String DIGEST_ALGORITHM = "SHA-256";
 
-    private static final int BUFFER_LENGTH = 1024;
+    private static final int BUFFER_LENGTH = 4096;
 
     private static final int START_READ_INDEX = 0;
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarUnpackMode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarUnpackMode.java
new file mode 100644
index 0000000000..d613632670
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarUnpackMode.java
@@ -0,0 +1,37 @@
+/*
+ * 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.nar;
+
+/**
+ * Specifies how the contents of a NAR file should be unpacked on disk.
+ */
+public enum NarUnpackMode {
+    /**
+     * Each JAR file in the NAR should be written out as a separate JAR file on disk.
+     * This is generally faster, but in order to use a URLClassLoader to load all JARs,
+     * many open file handles may be required.
+     */
+    UNPACK_INDIVIDUAL_JARS,
+
+    /**
+     * Each JAR file in the NAR should be combined into a single uber JAR file on disk.
+     * Unpacking to an uber jar is generally a slower process. However, it has the upside of
+     * being able to load all classes from the NAR using a single open file handle by the URLClassLoader.
+     */
+    UNPACK_TO_UBER_JAR;
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarUnpacker.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarUnpacker.java
index 63550ade86..9647a64489 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarUnpacker.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarUnpacker.java
@@ -24,6 +24,8 @@ import org.apache.nifi.util.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
 import java.io.BufferedReader;
 import java.io.File;
 import java.io.FileFilter;
@@ -31,6 +33,8 @@ import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.nio.charset.StandardCharsets;
 import java.nio.file.Files;
 import java.nio.file.Path;
 import java.nio.file.Paths;
@@ -47,6 +51,8 @@ import java.util.function.Predicate;
 import java.util.jar.Attributes;
 import java.util.jar.JarEntry;
 import java.util.jar.JarFile;
+import java.util.jar.JarInputStream;
+import java.util.jar.JarOutputStream;
 import java.util.jar.Manifest;
 
 import static java.lang.String.format;
@@ -63,28 +69,30 @@ public final class NarUnpacker {
         return nameToTest.endsWith(".nar") && pathname.isFile();
     };
 
-    public static ExtensionMapping unpackNars(final NiFiProperties props, final Bundle systemBundle) {
+    public static ExtensionMapping unpackNars(final NiFiProperties props, final Bundle systemBundle, final NarUnpackMode unpackMode) {
         // Default to NiFi's framework NAR ID if not given
-        return unpackNars(props, NarClassLoaders.FRAMEWORK_NAR_ID, systemBundle);
+        return unpackNars(props, NarClassLoaders.FRAMEWORK_NAR_ID, systemBundle, unpackMode);
     }
 
-    public static ExtensionMapping unpackNars(final NiFiProperties props, final String frameworkNarId, final Bundle systemBundle) {
+    public static ExtensionMapping unpackNars(final NiFiProperties props, final String frameworkNarId, final Bundle systemBundle, final NarUnpackMode unpackMode) {
         final List<Path> narLibraryDirs = props.getNarLibraryDirectories();
         final File frameworkWorkingDir = props.getFrameworkWorkingDirectory();
         final File extensionsWorkingDir = props.getExtensionsWorkingDirectory();
         final File docsWorkingDir = props.getComponentDocumentationWorkingDirectory();
 
-        return unpackNars(systemBundle, frameworkWorkingDir, frameworkNarId, extensionsWorkingDir, docsWorkingDir, narLibraryDirs);
+        return unpackNars(systemBundle, frameworkWorkingDir, frameworkNarId, extensionsWorkingDir, docsWorkingDir, narLibraryDirs, unpackMode);
     }
 
     public static ExtensionMapping unpackNars(final Bundle systemBundle, final File frameworkWorkingDir, final String frameworkNarId,
-                                              final File extensionsWorkingDir, final File docsWorkingDir, final List<Path> narLibraryDirs) {
-        return unpackNars(systemBundle, frameworkWorkingDir, extensionsWorkingDir, docsWorkingDir, narLibraryDirs, true, frameworkNarId, true, true, (coordinate) -> true);
+                                              final File extensionsWorkingDir, final File docsWorkingDir, final List<Path> narLibraryDirs,
+                                              final NarUnpackMode unpackMode) {
+        return unpackNars(systemBundle, frameworkWorkingDir, extensionsWorkingDir, docsWorkingDir, narLibraryDirs, true, frameworkNarId, true, true, unpackMode, (coordinate) -> true);
     }
 
     public static ExtensionMapping unpackNars(final Bundle systemBundle, final File frameworkWorkingDir, final File extensionsWorkingDir, final File docsWorkingDir, final List<Path> narLibraryDirs,
                                               final boolean requireFrameworkNar, final String frameworkNarId,
-                                              final boolean requireJettyNar, final boolean verifyHash, final Predicate<BundleCoordinate> narFilter) {
+                                              final boolean requireJettyNar, final boolean verifyHash, final NarUnpackMode unpackMode,
+                                              final Predicate<BundleCoordinate> narFilter) {
         final Map<File, BundleCoordinate> unpackedNars = new HashMap<>();
 
         try {
@@ -144,18 +152,18 @@ public final class NarUnpacker {
                             }
 
                             // unpack the framework nar
-                            unpackedFramework = unpackNar(narFile, frameworkWorkingDir, verifyHash);
+                            unpackedFramework = unpackNar(narFile, frameworkWorkingDir, verifyHash, unpackMode);
                         } else if (NarClassLoaders.JETTY_NAR_ID.equals(bundleCoordinate.getId())) {
                             if (unpackedJetty != null) {
                                 throw new IllegalStateException("Multiple Jetty NARs discovered. Only one Jetty NAR is permitted.");
                             }
 
                             // unpack and record the Jetty nar
-                            unpackedJetty = unpackNar(narFile, extensionsWorkingDir, verifyHash);
+                            unpackedJetty = unpackNar(narFile, extensionsWorkingDir, verifyHash, unpackMode);
                             unpackedExtensions.add(unpackedJetty);
                         } else {
                             // unpack and record the extension nar
-                            final File unpackedExtension = unpackNar(narFile, extensionsWorkingDir, verifyHash);
+                            final File unpackedExtension = unpackNar(narFile, extensionsWorkingDir, verifyHash, unpackMode);
                             unpackedExtensions.add(unpackedExtension);
                         }
                     }
@@ -216,10 +224,7 @@ public final class NarUnpacker {
 
             return extensionMapping;
         } catch (IOException e) {
-            logger.warn("Unable to load NAR library bundles due to " + e + " Will proceed without loading any further Nar bundles");
-            if (logger.isDebugEnabled()) {
-                logger.warn("", e);
-            }
+            logger.warn("Unable to load NAR library bundles due to {} Will proceed without loading any further Nar bundles", e.toString(), e);
         }
 
         return null;
@@ -293,15 +298,16 @@ public final class NarUnpacker {
      * @param verifyHash if the NAR has already been unpacked, indicates whether or not the hash should be verified. If this value is true,
      * and the NAR's hash does not match the hash written to the unpacked directory, the working directory will be deleted and the NAR will be
      * unpacked again. If false, the NAR will not be unpacked again and its hash will not be checked.
+     * @param unpackMode specifies how the contents of the NAR should be unpacked
      * @return the directory to the unpacked NAR
      * @throws IOException if unable to explode nar
      */
-    public static File unpackNar(final File nar, final File baseWorkingDirectory, final boolean verifyHash) throws IOException {
+    public static File unpackNar(final File nar, final File baseWorkingDirectory, final boolean verifyHash, final NarUnpackMode unpackMode) throws IOException {
         final File narWorkingDirectory = new File(baseWorkingDirectory, nar.getName() + "-unpacked");
 
         // if the working directory doesn't exist, unpack the nar
         if (!narWorkingDirectory.exists()) {
-            unpack(nar, narWorkingDirectory, FileDigestUtils.getDigest(nar));
+            unpackIndividualJars(nar, narWorkingDirectory, FileDigestUtils.getDigest(nar), unpackMode);
         } else if (verifyHash) {
             // the working directory does exist. Run digest against the nar
             // file and check if the nar has changed since it was deployed.
@@ -309,13 +315,13 @@ public final class NarUnpacker {
             final File workingHashFile = new File(narWorkingDirectory, HASH_FILENAME);
             if (!workingHashFile.exists()) {
                 FileUtils.deleteFile(narWorkingDirectory, true);
-                unpack(nar, narWorkingDirectory, narDigest);
+                unpackIndividualJars(nar, narWorkingDirectory, narDigest, unpackMode);
             } else {
                 final byte[] hashFileContents = Files.readAllBytes(workingHashFile.toPath());
                 if (!Arrays.equals(hashFileContents, narDigest)) {
                     logger.info("Contents of nar {} have changed. Reloading.", new Object[] { nar.getAbsolutePath() });
                     FileUtils.deleteFile(narWorkingDirectory, true);
-                    unpack(nar, narWorkingDirectory, narDigest);
+                    unpackIndividualJars(nar, narWorkingDirectory, narDigest, unpackMode);
                 }
             }
         } else {
@@ -325,24 +331,34 @@ public final class NarUnpacker {
         return narWorkingDirectory;
     }
 
+    private static void unpackIndividualJars(final File nar, final File workingDirectory, final byte[] hash, final NarUnpackMode unpackMode) throws IOException {
+        switch (unpackMode) {
+            case UNPACK_INDIVIDUAL_JARS:
+                unpackIndividualJars(nar, workingDirectory, hash);
+                return;
+            case UNPACK_TO_UBER_JAR:
+                unpackToUberJar(nar, workingDirectory, hash);
+        }
+    }
+
     /**
-     * Unpacks the NAR to the specified directory. Creates a checksum file that
+     * Unpacks the NAR to the specified directory. Creates a checksum file that can be
      * used to determine if future expansion is necessary.
      *
      * @param workingDirectory the root directory to which the NAR should be unpacked.
      * @throws IOException if the NAR could not be unpacked.
      */
-    private static void unpack(final File nar, final File workingDirectory, final byte[] hash) throws IOException {
-
-        try (JarFile jarFile = new JarFile(nar)) {
-            Enumeration<JarEntry> jarEntries = jarFile.entries();
+    private static void unpackIndividualJars(final File nar, final File workingDirectory, final byte[] hash) throws IOException {
+        try (final JarFile jarFile = new JarFile(nar)) {
+            final Enumeration<JarEntry> jarEntries = jarFile.entries();
             while (jarEntries.hasMoreElements()) {
-                JarEntry jarEntry = jarEntries.nextElement();
+                final JarEntry jarEntry = jarEntries.nextElement();
                 String name = jarEntry.getName();
-                if(name.contains("META-INF/bundled-dependencies")){
+                if (name.contains("META-INF/bundled-dependencies")){
                     name = name.replace("META-INF/bundled-dependencies", BUNDLED_DEPENDENCIES_DIRECTORY);
                 }
-                File f = new File(workingDirectory, name);
+
+                final File f = new File(workingDirectory, name);
                 if (jarEntry.isDirectory()) {
                     FileUtils.ensureDirectoryExistAndCanReadAndWrite(f);
                 } else {
@@ -357,6 +373,179 @@ public final class NarUnpacker {
         }
     }
 
+    /**
+     * Unpacks the NAR to a single JAR file in the specified directory. Creates a checksum file that can be
+     * used to determine if future expansion is necessary.
+     *
+     * @param workingDirectory the root directory to which the NAR should be unpacked.
+     * @throws IOException if the NAR could not be unpacked.
+     */
+    private static void unpackToUberJar(final File nar, final File workingDirectory, final byte[] hash) throws IOException {
+        logger.debug("====================================");
+        logger.debug("Unpacking NAR {}", nar.getAbsolutePath());
+
+        final File unpackedUberJarFile = new File(workingDirectory, "NAR-INF/bundled-dependencies/" + nar.getName() + ".unpacked.uber.jar");
+        Files.createDirectories(workingDirectory.toPath());
+        Files.createDirectories(unpackedUberJarFile.getParentFile().toPath());
+
+        final Set<String> entriesCreated = new HashSet<>();
+
+        try (final JarFile jarFile = new JarFile(nar);
+             final OutputStream out = new FileOutputStream(unpackedUberJarFile);
+             final OutputStream bufferedOut = new BufferedOutputStream(out);
+             final JarOutputStream uberJarOut = new JarOutputStream(bufferedOut)) {
+
+            final Enumeration<JarEntry> jarEntries = jarFile.entries();
+            while (jarEntries.hasMoreElements()) {
+                final JarEntry jarEntry = jarEntries.nextElement();
+                String name = jarEntry.getName();
+                if (name.contains("META-INF/bundled-dependencies")){
+                    name = name.replace("META-INF/bundled-dependencies", BUNDLED_DEPENDENCIES_DIRECTORY);
+                }
+
+                logger.debug("Unpacking NAR entry {}", name);
+
+                // If we've not yet created this entry, create it now. If we've already created the entry, ignore it.
+                if (!entriesCreated.add(name)) {
+                    continue;
+                }
+
+                // Explode anything from META-INF and any WAR files into the nar's output directory instead of copying it to the uber jar.
+                // The WAR files are important so that NiFi can load its UI. The META-INF/ directory is important in order to ensure that our
+                // NarClassLoader has all of the information that it needs.
+                if (name.contains("META-INF/") || (name.contains("NAR-INF") && name.endsWith(".war"))) {
+                    if (jarEntry.isDirectory()) {
+                        continue;
+                    }
+
+                    final File outFile = new File(workingDirectory, name);
+                    Files.createDirectories(outFile.getParentFile().toPath());
+
+                    try (final InputStream entryIn = jarFile.getInputStream(jarEntry);
+                         final OutputStream manifestOut = new FileOutputStream(outFile)) {
+                        copy(entryIn, manifestOut);
+                    }
+
+                    continue;
+                }
+
+                if (jarEntry.isDirectory()) {
+                    uberJarOut.putNextEntry(new JarEntry(jarEntry.getName()));
+                } else if (name.endsWith(".jar")) {
+                    // Unpack each .jar file into the uber jar, taking care to deal with META-INF/ files, etc. carefully.
+                    logger.debug("Unpacking Jar {}", name);
+
+                    try (final InputStream entryIn = jarFile.getInputStream(jarEntry);
+                         final InputStream in = new BufferedInputStream(entryIn)) {
+                        copyJarContents(in, uberJarOut, entriesCreated, workingDirectory);
+                    }
+                } else {
+                    // Copy the entry directly from NAR to the uber jar
+                    final JarEntry fileEntry = new JarEntry(jarEntry.getName());
+                    uberJarOut.putNextEntry(fileEntry);
+
+                    try (final InputStream entryIn = jarFile.getInputStream(jarEntry);
+                         final InputStream in = new BufferedInputStream(entryIn)) {
+                        copy(in, uberJarOut);
+                    }
+
+                    uberJarOut.closeEntry();
+                }
+            }
+        }
+
+        final File hashFile = new File(workingDirectory, HASH_FILENAME);
+        try (final FileOutputStream fos = new FileOutputStream(hashFile)) {
+            fos.write(hash);
+        }
+    }
+
+    /**
+     * Copies the contents of the Jar File whose input stream is provided to the JarOutputStream provided. Any META-INF files will be expanded into the
+     * appropriate location of the Working Directory. Other entries will be copied to the Jar Output Stream.
+     *
+     * @param in the InputStream from a jar file
+     * @param out the OutputStream to write the contents to
+     * @param entriesCreated the Set of all entries that have been created for the output Jar File. Any newly added entries will be added to this Set, so it must be mutable.
+     * @param workingDirectory the working directory for the nar
+     * @throws IOException if unable to copy the jar's entries
+     */
+    private static void copyJarContents(final InputStream in, final JarOutputStream out, final Set<String> entriesCreated, final File workingDirectory) throws IOException {
+        try (final JarInputStream jarInputStream = new JarInputStream(in)) {
+            JarEntry jarEntry;
+            while ((jarEntry = jarInputStream.getNextJarEntry()) != null) {
+                final String entryName = jarEntry.getName();
+
+                // The META-INF/ directory can contain several different types of files. For example, it contains:
+                // MANIFEST.MF
+                // LICENSE
+                // NOTICE
+                // Service Loader configuration
+                // Spring Handler configuration
+                //
+                // Of these, the License/Notice isn't particularly critical because this is a temporary file that's being created and loaded, not a file that is
+                // distributed. The Service Loader configurtion, Spring Handler, etc. can be dealt with by simply concatenating the contents together.
+                // But the MANIFEST.MF file is special. If it's not properly formed, it will prefer the ClassLoader from loading the JAR file, and we can't simply
+                // concatenate the files together. However, it's not required and generally contains information that we don't care about in this context. So we can
+                // simply ignore it.
+                if ((entryName.contains("META-INF/") && !entryName.contains("META-INF/MANIFEST.MF") ) && !jarEntry.isDirectory()) {
+                    logger.debug("Found META-INF/services file {}", entryName);
+
+                    final File outFile = new File(workingDirectory, entryName);
+
+                    // Because we're combining multiple jar files into one, we can run into situations where there may be conflicting filenames
+                    // such as 1 jar has a file named META-INF/license and another jar file has a META-INF/license/my-license.txt. We can generally
+                    // just ignore these, though, as they are not necessary in this temporarily created jar file. So we log it at a debug level and
+                    // move on.
+                    final File outDir = outFile.getParentFile();
+                    if (!outDir.exists() && !outDir.mkdirs()) {
+                        logger.debug("Skipping unpacking {} because parent file does not exist and could not be created", outFile);
+                        continue;
+                    }
+                    if (!outDir.isDirectory()) {
+                        logger.debug("Skipping unpacking {} because parent file is not a directory", outFile);
+                        continue;
+                    }
+
+                    // Write to file, appending to the existing file if it already exists.
+                    try (final OutputStream metaInfFileOut = new FileOutputStream(outFile, true);
+                         final OutputStream bufferedOut = new BufferedOutputStream(metaInfFileOut)) {
+                        copy(jarInputStream, bufferedOut);
+                        bufferedOut.write("\n".getBytes(StandardCharsets.UTF_8));
+                    }
+
+                    // Move to the next entry.
+                    continue;
+                }
+
+                // If the entry already exists, do not try to create another entry with the same name. Just skip the file.
+                if (!entriesCreated.add(entryName)) {
+                    logger.debug("Skipping entry {} in {} because an entry with that name already exists", entryName, workingDirectory);
+                    continue;
+                }
+
+                // Add a jar entry to the output JAR file and copy the contents of the file
+                final JarEntry outEntry = new JarEntry(jarEntry.getName());
+                out.putNextEntry(outEntry);
+
+                if (!jarEntry.isDirectory()) {
+                    copy(jarInputStream, out);
+                }
+
+                // Ensure that we close the entry.
+                out.closeEntry();
+            }
+        }
+    }
+
+    private static void copy(final InputStream in, final OutputStream out) throws IOException {
+        byte[] buffer = new byte[4096];
+        int len;
+        while ((len = in.read(buffer)) > 0) {
+            out.write(buffer, 0, len);
+        }
+    }
+
     private static void unpackDocumentation(final BundleCoordinate coordinate, final File jar, final File docsDirectory, final ExtensionMapping extensionMapping) throws IOException {
         final ExtensionMapping jarExtensionMapping = determineDocumentedNiFiComponents(coordinate, jar);
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/pom.xml
index 8bd65c08c4..96a7b823d1 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/pom.xml
@@ -79,6 +79,7 @@
         <nifi.nar.library.autoload.directory>./extensions</nifi.nar.library.autoload.directory>
         <nifi.nar.working.directory>./work/nar/</nifi.nar.working.directory>
         <nifi.documentation.working.directory>./work/docs/components</nifi.documentation.working.directory>
+        <nifi.nar.unpack.uber.jar>false</nifi.nar.unpack.uber.jar>
 
         <nifi.sensitive.props.key.protected />
         <nifi.sensitive.props.algorithm>NIFI_PBKDF2_AES_GCM_256</nifi.sensitive.props.algorithm>
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/nifi.properties b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/nifi.properties
index c0a593d23a..2d3fd6543a 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/nifi.properties
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/nifi.properties
@@ -39,6 +39,7 @@ nifi.nar.library.directory=${nifi.nar.library.directory}
 nifi.nar.library.autoload.directory=${nifi.nar.library.autoload.directory}
 nifi.nar.working.directory=${nifi.nar.working.directory}
 nifi.documentation.working.directory=${nifi.documentation.working.directory}
+nifi.nar.unpack.uber.jar=${nifi.nar.unpack.uber.jar}
 
 ####################
 # State Management #
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/main/java/org/apache/nifi/NiFi.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/main/java/org/apache/nifi/NiFi.java
index ad92500df9..79078bae40 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/main/java/org/apache/nifi/NiFi.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/main/java/org/apache/nifi/NiFi.java
@@ -23,6 +23,7 @@ import org.apache.nifi.nar.NarClassLoaders;
 import org.apache.nifi.nar.NarClassLoadersHolder;
 import org.apache.nifi.nar.NarUnpacker;
 import org.apache.nifi.nar.SystemBundle;
+import org.apache.nifi.nar.NarUnpackMode;
 import org.apache.nifi.processor.DataUnit;
 import org.apache.nifi.util.DiagnosticUtils;
 import org.apache.nifi.util.FileUtils;
@@ -137,7 +138,8 @@ public class NiFi implements NiFiEntryPoint {
         final Bundle systemBundle = SystemBundle.create(properties, rootClassLoader);
 
         // expand the nars
-        final ExtensionMapping extensionMapping = NarUnpacker.unpackNars(properties, systemBundle);
+        final NarUnpackMode unpackMode = properties.isUnpackNarsToUberJar() ? NarUnpackMode.UNPACK_TO_UBER_JAR : NarUnpackMode.UNPACK_INDIVIDUAL_JARS;
+        final ExtensionMapping extensionMapping = NarUnpacker.unpackNars(properties, systemBundle, unpackMode);
 
         // load the extensions classloaders
         NarClassLoaders narClassLoaders = NarClassLoadersHolder.getInstance();
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/main/java/org/apache/nifi/StatelessNiFi.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/main/java/org/apache/nifi/StatelessNiFi.java
index 4ae30dc255..c56e3f37be 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/main/java/org/apache/nifi/StatelessNiFi.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/main/java/org/apache/nifi/StatelessNiFi.java
@@ -17,6 +17,7 @@
 package org.apache.nifi;
 
 import org.apache.nifi.nar.NarUnpacker;
+import org.apache.nifi.nar.NarUnpackMode;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -69,7 +70,7 @@ public class StatelessNiFi {
             logger.info("Unpacking {} NARs", narFiles.length);
             final long startUnpack = System.nanoTime();
             for (final File narFile : narFiles) {
-                NarUnpacker.unpackNar(narFile, narWorkingDirectory, false);
+                NarUnpacker.unpackNar(narFile, narWorkingDirectory, false, NarUnpackMode.UNPACK_TO_UBER_JAR);
             }
 
             final long millis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startUnpack);
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/main/java/org/apache/nifi/web/server/JettyServer.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/main/java/org/apache/nifi/web/server/JettyServer.java
index 7063a7ecdc..054728b68c 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/main/java/org/apache/nifi/web/server/JettyServer.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/main/java/org/apache/nifi/web/server/JettyServer.java
@@ -51,6 +51,7 @@ import org.apache.nifi.nar.NarProvider;
 import org.apache.nifi.nar.NarThreadContextClassLoader;
 import org.apache.nifi.nar.StandardExtensionDiscoveringManager;
 import org.apache.nifi.nar.StandardNarLoader;
+import org.apache.nifi.nar.NarUnpackMode;
 import org.apache.nifi.security.util.TlsException;
 import org.apache.nifi.services.FlowService;
 import org.apache.nifi.ui.extension.UiExtension;
@@ -784,13 +785,15 @@ public class JettyServer implements NiFiServer, ExtensionUiLoader {
             DocGenerator.generate(props, extensionManager, extensionMapping);
 
             // Additionally loaded NARs and collected flow resources must be in place before starting the flows
+            final NarUnpackMode unpackMode = props.isUnpackNarsToUberJar() ? NarUnpackMode.UNPACK_TO_UBER_JAR : NarUnpackMode.UNPACK_INDIVIDUAL_JARS;
             final NarLoader narLoader = new StandardNarLoader(
                     props.getExtensionsWorkingDirectory(),
                     props.getComponentDocumentationWorkingDirectory(),
                     NarClassLoadersHolder.getInstance(),
                     extensionManager,
                     extensionMapping,
-                    this);
+                    this,
+                    unpackMode);
 
             narAutoLoader = new NarAutoLoader(props, narLoader);
             narAutoLoader.start();
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/accesscontrol/AccessControlHelper.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/accesscontrol/AccessControlHelper.java
index 2637cc2578..c03260f270 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/accesscontrol/AccessControlHelper.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/accesscontrol/AccessControlHelper.java
@@ -16,12 +16,6 @@
  */
 package org.apache.nifi.integration.accesscontrol;
 
-import static org.junit.Assert.assertEquals;
-
-import java.io.File;
-import java.nio.file.Files;
-import java.nio.file.StandardCopyOption;
-import javax.ws.rs.core.Response;
 import org.apache.nifi.bundle.Bundle;
 import org.apache.nifi.integration.NiFiWebApiTest;
 import org.apache.nifi.integration.util.NiFiTestAuthorizer;
@@ -33,9 +27,17 @@ import org.apache.nifi.nar.NarClassLoadersHolder;
 import org.apache.nifi.nar.NarUnpacker;
 import org.apache.nifi.nar.StandardExtensionDiscoveringManager;
 import org.apache.nifi.nar.SystemBundle;
+import org.apache.nifi.nar.NarUnpackMode;
 import org.apache.nifi.util.NiFiProperties;
 import org.apache.nifi.util.StringUtils;
 
+import javax.ws.rs.core.Response;
+import java.io.File;
+import java.nio.file.Files;
+import java.nio.file.StandardCopyOption;
+
+import static org.junit.Assert.assertEquals;
+
 /**
  * Access control test for the dfm user.
  */
@@ -81,7 +83,8 @@ public class AccessControlHelper {
         }
 
         final Bundle systemBundle = SystemBundle.create(props);
-        NarUnpacker.unpackNars(props, systemBundle);
+
+        NarUnpacker.unpackNars(props, systemBundle, NarUnpackMode.UNPACK_INDIVIDUAL_JARS);
         NarClassLoadersHolder.getInstance().init(props.getFrameworkWorkingDirectory(), props.getExtensionsWorkingDirectory());
 
         // load extensions
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/accesscontrol/OneWaySslAccessControlHelper.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/accesscontrol/OneWaySslAccessControlHelper.java
index 6faf991d02..aba47985d9 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/accesscontrol/OneWaySslAccessControlHelper.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/accesscontrol/OneWaySslAccessControlHelper.java
@@ -16,10 +16,6 @@
  */
 package org.apache.nifi.integration.accesscontrol;
 
-import java.io.File;
-import java.nio.file.Files;
-import java.nio.file.StandardCopyOption;
-import javax.ws.rs.client.Client;
 import org.apache.commons.io.FileUtils;
 import org.apache.nifi.bundle.Bundle;
 import org.apache.nifi.integration.util.NiFiTestServer;
@@ -30,12 +26,18 @@ import org.apache.nifi.nar.NarClassLoadersHolder;
 import org.apache.nifi.nar.NarUnpacker;
 import org.apache.nifi.nar.StandardExtensionDiscoveringManager;
 import org.apache.nifi.nar.SystemBundle;
+import org.apache.nifi.nar.NarUnpackMode;
 import org.apache.nifi.security.util.SslContextFactory;
 import org.apache.nifi.security.util.StandardTlsConfiguration;
 import org.apache.nifi.security.util.TlsConfiguration;
 import org.apache.nifi.util.NiFiProperties;
 import org.apache.nifi.web.util.WebUtils;
 
+import javax.ws.rs.client.Client;
+import java.io.File;
+import java.nio.file.Files;
+import java.nio.file.StandardCopyOption;
+
 /**
  * Access control test for the dfm user.
  */
@@ -74,7 +76,8 @@ public class OneWaySslAccessControlHelper {
         }
 
         final Bundle systemBundle = SystemBundle.create(props);
-        NarUnpacker.unpackNars(props, systemBundle);
+
+        NarUnpacker.unpackNars(props, systemBundle, NarUnpackMode.UNPACK_INDIVIDUAL_JARS);
         NarClassLoadersHolder.getInstance().init(props.getFrameworkWorkingDirectory(), props.getExtensionsWorkingDirectory());
 
         // load extensions
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 78187454fa..1314d6694a 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
@@ -23,6 +23,7 @@ import org.apache.nifi.nar.NarClassLoader;
 import org.apache.nifi.nar.NarClassLoaders;
 import org.apache.nifi.nar.NarUnpacker;
 import org.apache.nifi.nar.SystemBundle;
+import org.apache.nifi.nar.NarUnpackMode;
 import org.apache.nifi.stateless.config.ParameterOverride;
 import org.apache.nifi.stateless.config.StatelessConfigurationException;
 import org.apache.nifi.stateless.engine.NarUnpackLock;
@@ -109,7 +110,10 @@ public class StatelessBootstrap {
         final Predicate<BundleCoordinate> narFilter = coordinate -> true;
         NarUnpackLock.lock();
         try {
-            NarUnpacker.unpackNars(systemBundle, frameworkWorkingDir, extensionsWorkingDir, null, narDirectories, false, NarClassLoaders.FRAMEWORK_NAR_ID, false, false, narFilter);
+            // For many environments where Stateless is to be run, the number of open file handles may be constrained. Because of this,
+            // we will unpack NARs using the Uber Jar method.
+            NarUnpacker.unpackNars(systemBundle, frameworkWorkingDir, extensionsWorkingDir, null, narDirectories, false, NarClassLoaders.FRAMEWORK_NAR_ID, false, false,
+                NarUnpackMode.UNPACK_TO_UBER_JAR, narFilter);
         } finally {
             NarUnpackLock.unlock();
         }
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 45df96cfef..a6499ad5e3 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
@@ -24,6 +24,7 @@ import org.apache.nifi.nar.ExtensionDiscoveringManager;
 import org.apache.nifi.nar.NarClassLoaders;
 import org.apache.nifi.nar.NarLoadResult;
 import org.apache.nifi.nar.NarUnpacker;
+import org.apache.nifi.nar.NarUnpackMode;
 import org.apache.nifi.stateless.engine.NarUnpackLock;
 import org.apache.nifi.stateless.engine.StatelessEngineConfiguration;
 import org.slf4j.Logger;
@@ -153,7 +154,7 @@ public class FileSystemExtensionRepository implements ExtensionRepository {
             try {
                 logger.info("Unpacking {}", downloadedFile);
                 final File extensionsWorkingDirectory = new File(workingDirectory, "extensions");
-                final File unpackedDir = NarUnpacker.unpackNar(downloadedFile, extensionsWorkingDirectory, false);
+                final File unpackedDir = NarUnpacker.unpackNar(downloadedFile, extensionsWorkingDirectory, false, NarUnpackMode.UNPACK_TO_UBER_JAR);
                 unpackedDirs.add(unpackedDir);
             } finally {
                 NarUnpackLock.unlock();