You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by bb...@apache.org on 2020/10/07 13:12:38 UTC

[nifi] branch main updated: NIFI-7592: Allow NiFi to be started without a GUI/REST interface (#4509)

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

bbende 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 325a495  NIFI-7592: Allow NiFi to be started without a GUI/REST interface (#4509)
325a495 is described below

commit 325a49556c6cb2effbdb936889ebf45b212d2a46
Author: Matthew Burgess <ma...@apache.org>
AuthorDate: Wed Oct 7 09:12:16 2020 -0400

    NIFI-7592: Allow NiFi to be started without a GUI/REST interface (#4509)
    
    * NIFI-7592: Allow NiFi to be started without a GUI/REST interface
    
    * NIFI-7592: Enable all controller services when starting headless
    
    * NIFI-7592: Marked duplicate dependencies as provided
    
    * NIFI-7592: Incorporated additional review comments
---
 nifi-assembly/pom.xml                              |  33 +
 .../nifi-framework-nar/pom.xml                     |  14 +-
 .../nifi-framework/nifi-documentation/pom.xml      |   4 +
 .../documentation/example/NiFiServerStub.java}     |  34 +-
 .../src/test/resources/lib/nifiserver-test-nar.nar | Bin 0 -> 298794 bytes
 .../org/apache/nifi/authorization/FlowParser.java  |  80 +-
 .../java/org/apache/nifi/groups/ProcessGroup.java  |   6 +
 .../apache/nifi/groups/StandardProcessGroup.java   |  12 +
 .../controller/service/mock/MockProcessGroup.java  |   5 +
 .../nifi-framework-nar-loading-utils/pom.xml       |   4 +
 .../org/apache/nifi/nar/StandardNarLoader.java     |   4 +-
 .../org/apache/nifi/nar/AbstractTestNarLoader.java |   4 +-
 .../java/org/apache/nifi/nar/TestNarLoader.java    |   4 +-
 .../src/test/resources/lib/nifiserver-test-nar.nar | Bin 0 -> 298794 bytes
 .../nifi-framework-nar-utils/pom.xml               |   5 +
 .../nifi/nar/NarThreadContextClassLoader.java      |   1 -
 .../nifi-framework/nifi-headless-server/pom.xml    |  90 +++
 .../org/apache/nifi/headless/FlowEnricher.java     | 264 +++++++
 .../apache/nifi/headless/HeadlessNiFiServer.java   | 236 ++++++
 .../META-INF/services/org.apache.nifi.NiFiServer   |  15 +
 .../nifi-framework/nifi-nar-utils/pom.xml          |   4 +
 .../java/org/apache/nifi/nar/NarClassLoaders.java  |  68 +-
 .../main/java/org/apache/nifi/nar/NarUnpacker.java |   9 +-
 .../nifi-framework/nifi-runtime/pom.xml            |   5 +
 .../java/org/apache/nifi/BootstrapListener.java    |  24 +-
 .../src/main/java/org/apache/nifi/NiFi.java        |  29 +-
 .../nifi/{NiFiServer.java => NiFiEntryPoint.java}  |  23 +-
 .../FlowEnrichmentException.java}                  |  44 +-
 .../nifi-framework/nifi-web/nifi-jetty/pom.xml     |  40 +-
 .../org/apache/nifi/web/server/JettyServer.java    |  35 +-
 .../META-INF/services/org.apache.nifi.NiFiServer   |  15 +
 .../nifi/web/server/JettyServerGroovyTest.groovy   |   5 +-
 .../nifi-framework/nifi-web/nifi-web-api/pom.xml   |   5 +
 .../nifi-framework/nifi-web/nifi-web-docs/pom.xml  |   5 +
 .../nifi-framework-bundle/nifi-framework/pom.xml   |   1 +
 .../pom.xml                                        |  41 +-
 .../src/main/resources/META-INF/LICENSE            | 313 ++++++++
 .../src/main/resources/META-INF/NOTICE             | 234 ++++++
 .../nifi-framework-bundle/nifi-server-nar/pom.xml  | 198 +++++
 .../src/main/resources/META-INF/LICENSE            | 880 +++++++++++++++++++++
 .../src/main/resources/META-INF/NOTICE             | 234 ++++++
 nifi-nar-bundles/nifi-framework-bundle/pom.xml     |   2 +
 .../nifi-update-attribute-nar/pom.xml              |   6 +
 nifi-nar-bundles/pom.xml                           |   6 +
 nifi-server-api/pom.xml                            |  46 ++
 .../src/main/java/org/apache/nifi/NiFiServer.java  |   5 +-
 .../java/org/apache/nifi/nar/ExtensionMapping.java |   0
 pom.xml                                            |   1 +
 48 files changed, 2946 insertions(+), 147 deletions(-)

diff --git a/nifi-assembly/pom.xml b/nifi-assembly/pom.xml
index 38fa5b4..2edb536 100644
--- a/nifi-assembly/pom.xml
+++ b/nifi-assembly/pom.xml
@@ -103,6 +103,11 @@ language governing permissions and limitations under the License. -->
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-server-api</artifactId>
+            <version>1.13.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-runtime</artifactId>
             <version>1.13.0-SNAPSHOT</version>
         </dependency>
@@ -135,6 +140,12 @@ language governing permissions and limitations under the License. -->
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-server-nar</artifactId>
+            <version>1.13.0-SNAPSHOT</version>
+            <type>nar</type>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-provenance-repository-nar</artifactId>
             <version>1.13.0-SNAPSHOT</version>
             <type>nar</type>
@@ -952,6 +963,28 @@ language governing permissions and limitations under the License. -->
                 </dependency>
             </dependencies>
     	</profile>
+        <profile>
+            <id>headless</id>
+            <!-- This profile excludes the nifi-server artifacts, such as the Web/UI NAR(s) and instead uses the nifi-headless-server-nar. -->
+            <activation>
+                <activeByDefault>false</activeByDefault>
+            </activation>
+            <dependencies>
+                <dependency>
+                    <groupId>org.apache.nifi</groupId>
+                    <artifactId>nifi-server-nar</artifactId>
+                    <version>1.13.0-SNAPSHOT</version>
+                    <type>nar</type>
+                    <scope>provided</scope>
+                </dependency>
+                <dependency>
+                    <groupId>org.apache.nifi</groupId>
+                    <artifactId>nifi-headless-server-nar</artifactId>
+                    <version>1.13.0-SNAPSHOT</version>
+                    <type>nar</type>
+                </dependency>
+            </dependencies>
+        </profile>
 	<profile>
             <id>include-accumulo</id>
             <!-- This profile handles the inclusion of nifi-accumulo artifacts. -->
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework-nar/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework-nar/pom.xml
index 142cf18..5aa1109 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework-nar/pom.xml
@@ -29,7 +29,9 @@
     <dependencies>
         <dependency>
             <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-jetty</artifactId>
+            <artifactId>nifi-jetty-bundle</artifactId>
+            <version>1.13.0-SNAPSHOT</version>
+            <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
@@ -50,9 +52,19 @@
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-framework-cluster</artifactId>
+            <version>1.13.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-stateless</artifactId>
             <version>1.13.0-SNAPSHOT</version>
         </dependency>
+        <dependency>
+            <groupId>org.glassfish.jersey.core</groupId>
+            <artifactId>jersey-server</artifactId>
+            <version>${jersey.version}</version>
+        </dependency>
         <!-- Override scope to compile since framework NAR won't get this from a parent NAR -->
         <dependency>
             <groupId>org.apache.nifi</groupId>
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/pom.xml
index 12e338d..2296d04 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/pom.xml
@@ -28,6 +28,10 @@
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-server-api</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-properties</artifactId>
         </dependency>
         <dependency>
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/main/java/org/apache/nifi/NiFiServer.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/example/NiFiServerStub.java
similarity index 51%
copy from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/main/java/org/apache/nifi/NiFiServer.java
copy to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/example/NiFiServerStub.java
index 2169579..f77b909 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/main/java/org/apache/nifi/NiFiServer.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/example/NiFiServerStub.java
@@ -14,28 +14,44 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.nifi;
+package org.apache.nifi.documentation.example;
 
+import org.apache.nifi.NiFiServer;
 import org.apache.nifi.bundle.Bundle;
 import org.apache.nifi.diagnostics.DiagnosticsFactory;
 import org.apache.nifi.nar.ExtensionMapping;
+import org.apache.nifi.util.NiFiProperties;
 
 import java.util.Set;
 
 /**
- *
+ * This stub is the source code for the no-op NiFiServer implementation used in the nifiserver-test-nar.nar, as NiFi requires exactly one
+ * implementation of NiFiServer in order to start successfully. The NAR was built externally, but the code is provided here in case
+ * updates are needed.
  */
-public interface NiFiServer {
+public class NiFiServerStub implements NiFiServer {
+    @Override
+    public void start() {
+
+    }
 
-    void start();
+    @Override
+    public void initialize(NiFiProperties properties, Bundle systemBundle, Set<Bundle> bundles, ExtensionMapping extensionMapping) {
 
-    void setExtensionMapping(ExtensionMapping extensionMapping);
+    }
 
-    void setBundles(Bundle systemBundle, Set<Bundle> bundles);
+    @Override
+    public void stop() {
 
-    void stop();
+    }
 
-    DiagnosticsFactory getDiagnosticsFactory();
+    @Override
+    public DiagnosticsFactory getDiagnosticsFactory() {
+        return null;
+    }
 
-    DiagnosticsFactory getThreadDumpFactory();
+    @Override
+    public DiagnosticsFactory getThreadDumpFactory() {
+        return null;
+    }
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/resources/lib/nifiserver-test-nar.nar b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/resources/lib/nifiserver-test-nar.nar
new file mode 100644
index 0000000..c60026e
Binary files /dev/null and b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/resources/lib/nifiserver-test-nar.nar differ
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/main/java/org/apache/nifi/authorization/FlowParser.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/main/java/org/apache/nifi/authorization/FlowParser.java
index b5ca365..b2145d5 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/main/java/org/apache/nifi/authorization/FlowParser.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/main/java/org/apache/nifi/authorization/FlowParser.java
@@ -17,24 +17,34 @@
 package org.apache.nifi.authorization;
 
 import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
 import java.io.File;
 import java.io.IOException;
 import java.io.InputStream;
+import java.io.OutputStream;
 import java.nio.file.Files;
 import java.nio.file.Path;
 import java.nio.file.StandardOpenOption;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.zip.GZIPInputStream;
+import java.util.zip.GZIPOutputStream;
 import javax.xml.XMLConstants;
 import javax.xml.parsers.DocumentBuilder;
 import javax.xml.parsers.ParserConfigurationException;
+import javax.xml.transform.Result;
+import javax.xml.transform.Source;
+import javax.xml.transform.TransformerException;
+import javax.xml.transform.TransformerFactory;
+import javax.xml.transform.dom.DOMSource;
+import javax.xml.transform.stream.StreamResult;
 import javax.xml.validation.Schema;
 import javax.xml.validation.SchemaFactory;
 import org.apache.commons.io.IOUtils;
 import org.apache.nifi.controller.serialization.FlowFromDOMFactory;
 import org.apache.nifi.security.xml.XmlUtils;
 import org.apache.nifi.util.LoggingXmlParserErrorHandler;
+import org.apache.nifi.util.file.FileUtils;
 import org.apache.nifi.web.api.dto.PortDTO;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -54,10 +64,9 @@ public class FlowParser {
     private static final String FLOW_XSD = "/FlowConfiguration.xsd";
 
     private final Schema flowSchema;
-    private final SchemaFactory schemaFactory;
 
     public FlowParser() throws SAXException {
-        schemaFactory = SchemaFactory.newInstance(XMLConstants.W3C_XML_SCHEMA_NS_URI);
+        SchemaFactory schemaFactory = SchemaFactory.newInstance(XMLConstants.W3C_XML_SCHEMA_NS_URI);
         flowSchema = schemaFactory.newSchema(FlowParser.class.getResource(FLOW_XSD));
     }
 
@@ -131,6 +140,48 @@ public class FlowParser {
     }
 
     /**
+     * Generates a {@link Document} from the flow configuration file provided
+     */
+    public Document parseDocument(final File flowConfigurationFile) {
+        if (flowConfigurationFile == null) {
+            logger.debug("Flow Configuration file was null");
+            return null;
+        }
+
+        // if the flow doesn't exist or is 0 bytes, then return null
+        final Path flowPath = flowConfigurationFile.toPath();
+        try {
+            if (!Files.exists(flowPath) || Files.size(flowPath) == 0) {
+                logger.warn("Flow Configuration does not exist or was empty");
+                return null;
+            }
+        } catch (IOException e) {
+            logger.error("An error occurred determining the size of the Flow Configuration file");
+            return null;
+        }
+
+        // otherwise create the appropriate input streams to read the file
+        try (final InputStream in = Files.newInputStream(flowPath, StandardOpenOption.READ);
+             final InputStream gzipIn = new GZIPInputStream(in)) {
+
+            final byte[] flowBytes = IOUtils.toByteArray(gzipIn);
+            if (flowBytes == null || flowBytes.length == 0) {
+                logger.warn("Could not extract root group id because Flow Configuration File was empty");
+                return null;
+            }
+
+            // create validating document builder
+            final DocumentBuilder docBuilder = XmlUtils.createSafeDocumentBuilder(flowSchema);
+            docBuilder.setErrorHandler(new LoggingXmlParserErrorHandler("Flow Configuration", logger));
+            return docBuilder.parse(new ByteArrayInputStream(flowBytes));
+
+        } catch (final SAXException | ParserConfigurationException | IOException ex) {
+            logger.error("Unable to parse flow {} due to {}", new Object[]{flowPath.toAbsolutePath(), ex});
+            return null;
+        }
+    }
+
+    /**
      * Gets the ports that are direct children of the given element.
      *
      * @param element the element containing ports
@@ -152,13 +203,34 @@ public class FlowParser {
     }
 
     /**
+     * Writes a given XML Flow out to the specified path.
+     *
+     * @param flowDocument flowDocument of the associated XML content to write to disk
+     * @param flowXmlPath  path on disk to write the flow
+     * @throws IOException if there are issues in accessing the target destination for the flow
+     * @throws TransformerException if there are issues in the xml transformation process
+     */
+    public void writeFlow(final Document flowDocument, final Path flowXmlPath) throws IOException, TransformerException {
+        final ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
+        final Source xmlSource = new DOMSource(flowDocument);
+        final Result outputTarget = new StreamResult(outputStream);
+        TransformerFactory.newInstance().newTransformer().transform(xmlSource, outputTarget);
+        final InputStream is = new ByteArrayInputStream(outputStream.toByteArray());
+
+        try (final OutputStream output = Files.newOutputStream(flowXmlPath, StandardOpenOption.WRITE, StandardOpenOption.CREATE);
+             final OutputStream gzipOut = new GZIPOutputStream(output)) {
+            FileUtils.copy(is, gzipOut);
+        }
+    }
+
+    /**
      * Finds child elements with the given tagName.
      *
      * @param element the parent element
      * @param tagName the child element name to find
      * @return a list of matching child elements
      */
-    private static List<Element> getChildrenByTagName(final Element element, final String tagName) {
+    public static List<Element> getChildrenByTagName(final Element element, final String tagName) {
         final List<Element> matches = new ArrayList<>();
         final NodeList nodeList = element.getChildNodes();
         for (int i = 0; i < nodeList.getLength(); i++) {
@@ -172,8 +244,6 @@ public class FlowParser {
                 matches.add(child);
             }
         }
-
         return matches;
     }
-
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/ProcessGroup.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/ProcessGroup.java
index 2656792..d724c46 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/ProcessGroup.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/ProcessGroup.java
@@ -187,6 +187,12 @@ public interface ProcessGroup extends ComponentAuthorizable, Positionable, Versi
     void enableOutputPort(Port port);
 
     /**
+     * Recursively enables all Controller Services for this Process Group and all child Process Groups
+     *
+     */
+    void enableAllControllerServices();
+
+    /**
      * Starts the given Processor
      *
      * @param processor the processor to start
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java
index 0bd0299..1345791 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java
@@ -478,6 +478,7 @@ public final class StandardProcessGroup implements ProcessGroup {
     public void startProcessing() {
         readLock.lock();
         try {
+            enableAllControllerServices();
             findAllProcessors().stream().filter(START_PROCESSORS_FILTER).forEach(node -> {
                 try {
                     node.getProcessGroup().startProcessor(node, true);
@@ -1684,6 +1685,17 @@ public final class StandardProcessGroup implements ProcessGroup {
     }
 
     @Override
+    public void enableAllControllerServices() {
+        // Enable all valid controller services in this process group
+        controllerServiceProvider.enableControllerServices(controllerServices.values());
+
+        // Enable all controller services for child process groups
+        for(ProcessGroup pg : processGroups.values()) {
+            pg.enableAllControllerServices();
+        }
+    }
+
+    @Override
     public void disableInputPort(final Port port) {
         readLock.lock();
         try {
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/mock/MockProcessGroup.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/mock/MockProcessGroup.java
index 15613af..0e0e7e8 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/mock/MockProcessGroup.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/mock/MockProcessGroup.java
@@ -165,6 +165,11 @@ public class MockProcessGroup implements ProcessGroup {
     }
 
     @Override
+    public void enableAllControllerServices() {
+
+    }
+
+    @Override
     public CompletableFuture<Void> startProcessor(final ProcessorNode processor, final boolean failIfStopping) {
         return CompletableFuture.completedFuture(null);
     }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/pom.xml
index 578252e..682cc52 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/pom.xml
@@ -43,6 +43,10 @@
             <artifactId>nifi-framework-api</artifactId>
         </dependency>
         <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-server-api</artifactId>
+        </dependency>
+        <dependency>
             <groupId>org.apache.commons</groupId>
             <artifactId>commons-lang3</artifactId>
             <version>3.9</version>
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 20b566b..99ef8f3 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
@@ -124,7 +124,9 @@ public class StandardNarLoader implements NarLoader {
             }
 
             LOGGER.debug("Loading custom UIs for extensions...");
-            extensionUiLoader.loadExtensionUis(loadedBundles);
+            if(extensionUiLoader != null) {
+                extensionUiLoader.loadExtensionUis(loadedBundles);
+            }
         }
 
         LOGGER.info("Finished NAR loading process!");
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 22fd133..481eae7 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
@@ -80,8 +80,8 @@ public abstract class AbstractTestNarLoader {
 
         extensionManager = new StandardExtensionDiscoveringManager();
 
-        // Should have Framework and Jetty NARs loaded here
-        assertEquals(2, narClassLoaders.getBundles().size());
+        // Should have Framework, Jetty, and NiFiServer NARs loaded here
+        assertEquals(3, narClassLoaders.getBundles().size());
 
         // No extensions should be loaded yet
         assertEquals(0, extensionManager.getExtensions(Processor.class).size());
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/test/java/org/apache/nifi/nar/TestNarLoader.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/test/java/org/apache/nifi/nar/TestNarLoader.java
index 94a08d6..4f09402 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/test/java/org/apache/nifi/nar/TestNarLoader.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/test/java/org/apache/nifi/nar/TestNarLoader.java
@@ -64,7 +64,7 @@ public class TestNarLoader extends AbstractTestNarLoader {
         assertEquals(3, narLoadResult.getLoadedBundles().size());
         assertEquals(0, narLoadResult.getSkippedBundles().size());
 
-        assertEquals(5, narClassLoaders.getBundles().size());
+        assertEquals(6, narClassLoaders.getBundles().size());
         assertEquals(1, extensionManager.getExtensions(Processor.class).size());
         assertEquals(1, extensionManager.getExtensions(ControllerService.class).size());
         assertEquals(0, extensionManager.getExtensions(ReportingTask.class).size());
@@ -111,7 +111,7 @@ public class TestNarLoader extends AbstractTestNarLoader {
         assertEquals(3, narLoadResult3.getLoadedBundles().size());
         assertEquals(0, narLoadResult3.getSkippedBundles().size());
 
-        assertEquals(5, narClassLoaders.getBundles().size());
+        assertEquals(6, narClassLoaders.getBundles().size());
         assertEquals(1, extensionManager.getExtensions(Processor.class).size());
         assertEquals(1, extensionManager.getExtensions(ControllerService.class).size());
         assertEquals(0, extensionManager.getExtensions(ReportingTask.class).size());
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/test/resources/lib/nifiserver-test-nar.nar b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/test/resources/lib/nifiserver-test-nar.nar
new file mode 100644
index 0000000..c60026e
Binary files /dev/null and b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/test/resources/lib/nifiserver-test-nar.nar differ
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/pom.xml
index 5c88078..fc5d2e9 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/pom.xml
@@ -38,5 +38,10 @@
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-framework-api</artifactId>
         </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-server-api</artifactId>
+            <scope>test</scope>
+        </dependency>
     </dependencies>
 </project>
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/NarThreadContextClassLoader.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/NarThreadContextClassLoader.java
index 241bcd5..7b3e020 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/NarThreadContextClassLoader.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/NarThreadContextClassLoader.java
@@ -192,7 +192,6 @@ public class NarThreadContextClassLoader extends URLClassLoader {
     public static <T> T createInstance(final ExtensionManager extensionManager, final String implementationClassName, final Class<T> typeDefinition, final NiFiProperties nifiProperties)
             throws InstantiationException, IllegalAccessException, ClassNotFoundException {
         final ClassLoader originalClassLoader = Thread.currentThread().getContextClassLoader();
-        Thread.currentThread().setContextClassLoader(NarThreadContextClassLoader.getInstance());
         try {
             final List<Bundle> bundles = extensionManager.getBundles(implementationClassName);
             if (bundles.size() == 0) {
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-headless-server/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-headless-server/pom.xml
new file mode 100644
index 0000000..395284a
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-headless-server/pom.xml
@@ -0,0 +1,90 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+      http://www.apache.org/licenses/LICENSE-2.0
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>nifi-framework</artifactId>
+        <version>1.13.0-SNAPSHOT</version>
+    </parent>
+    <artifactId>nifi-headless-server</artifactId>
+    <packaging>jar</packaging>
+    <dependencies>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-log4j12</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-runtime</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-framework-nar-loading-utils</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-nar-utils</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-properties</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-framework-core</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-framework-cluster</artifactId>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-file-authorizer</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.commons</groupId>
+            <artifactId>commons-text</artifactId>
+            <version>1.8</version>
+        </dependency>
+        <dependency>
+            <groupId>com.github.stefanbirkner</groupId>
+            <artifactId>system-rules</artifactId>
+            <version>1.19.0</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.springframework</groupId>
+            <artifactId>spring-test</artifactId>
+            <version>5.0.9.RELEASE</version>
+            <scope>test</scope>
+        </dependency>
+    </dependencies>
+</project>
+
+
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-headless-server/src/main/java/org/apache/nifi/headless/FlowEnricher.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-headless-server/src/main/java/org/apache/nifi/headless/FlowEnricher.java
new file mode 100644
index 0000000..c3503ee
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-headless-server/src/main/java/org/apache/nifi/headless/FlowEnricher.java
@@ -0,0 +1,264 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.headless;
+
+import org.apache.nifi.authorization.FlowParser;
+import org.apache.nifi.bundle.Bundle;
+import org.apache.nifi.bundle.BundleCoordinate;
+import org.apache.nifi.bundle.BundleDetails;
+import org.apache.nifi.util.NiFiProperties;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.w3c.dom.Document;
+import org.w3c.dom.Element;
+import org.w3c.dom.Node;
+import org.w3c.dom.NodeList;
+
+import javax.xml.transform.TransformerException;
+import java.io.IOException;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+public class FlowEnricher {
+
+    private static final Logger logger = LoggerFactory.getLogger(FlowEnricher.class);
+
+    private final HeadlessNiFiServer headlessNiFiServer;
+    private final FlowParser flowParser;
+    private final NiFiProperties niFiProperties;
+
+    public static final String PROCESSOR_TAG_NAME = "processor";
+    public static final String CONTROLLER_SERVICE_TAG_NAME = "controllerService";
+    public static final String REPORTING_TASK_TAG_NAME = "reportingTask";
+
+    public FlowEnricher(HeadlessNiFiServer headlessNiFiServer, FlowParser flowParser, NiFiProperties niFiProperties) {
+        this.headlessNiFiServer = headlessNiFiServer;
+        this.flowParser = flowParser;
+        this.niFiProperties = niFiProperties;
+    }
+
+    /**
+     * Traverse a flow document and enrich all components with bundle pairings that satisfy the constraints presented by
+     * the versions of bundles supplied on the classpath.
+     * <p>
+     * The primary nature of these relationships is comprised of a standlone instance
+     *
+     * @throws FlowEnrichmentException if the provided flow cannot be enriched
+     */
+    public void enrichFlowWithBundleInformation() throws FlowEnrichmentException {
+        final Path flowPath = niFiProperties.getFlowConfigurationFile().toPath();
+        logger.debug("Enriching generated {} with bundling information", flowPath.toAbsolutePath());
+
+        try {
+            // Prepare elements and establish initial bookkeeping to use for analysis
+            final Document flowDocument = flowParser.parseDocument(flowPath.toAbsolutePath().toFile());
+
+            if (flowDocument == null) {
+                throw new FlowEnrichmentException("Unable to successfully parse the specified flow at " + flowPath.toAbsolutePath());
+            }
+
+            // Aggregate all dependency mappings of all component types that need to have a bundle evaluated with their
+            // associated XML information
+            final Map<String, EnrichingElementAdapter> componentEnrichingMap = new HashMap<>();
+
+            // Treat all component types as one map
+            for (String typeElementName : Arrays.asList(PROCESSOR_TAG_NAME, CONTROLLER_SERVICE_TAG_NAME, REPORTING_TASK_TAG_NAME)) {
+                final NodeList componentNodeList = flowDocument.getElementsByTagName(typeElementName);
+                final Map<String, EnrichingElementAdapter> elementIdToMetadataMap = mapComponents(componentNodeList);
+
+                componentEnrichingMap.putAll(elementIdToMetadataMap);
+
+            }
+
+            // For each of the components we have, evaluate its dependencies and apply versions
+            for (Map.Entry<String, EnrichingElementAdapter> componentIdToMetadata : componentEnrichingMap.entrySet()) {
+                // If this particular component has already had bundle information applied, skip it
+                final EnrichingElementAdapter componentToEnrich = componentIdToMetadata.getValue();
+                if (componentToEnrich.getBundleElement() != null) {
+                    continue;
+                }
+
+                final String componentToEnrichClass = componentToEnrich.getComponentClass();
+                final Map<String, Bundle> componentToEnrichVersionToBundles = headlessNiFiServer.getBundles(componentToEnrichClass)
+                        .stream()
+                        .collect(Collectors.toMap(bundle -> bundle.getBundleDetails().getCoordinate().getVersion(), bundle -> bundle));
+
+                enrichComponent(componentToEnrich, componentToEnrichVersionToBundles);
+                // verify error conditions
+            }
+
+            flowParser.writeFlow(flowDocument, flowPath.toAbsolutePath());
+        } catch (IOException | TransformerException e) {
+            throw new FlowEnrichmentException("Unable to successfully automate the enrichment of the generated flow with bundle information", e);
+        }
+    }
+
+    private void enrichComponent(EnrichingElementAdapter componentToEnrich, Map<String, Bundle> componentToEnrichVersionToBundles) throws FlowEnrichmentException {
+
+        if (componentToEnrich.getBundleElement() != null) {
+            return;
+        }
+
+        BundleCoordinate enrichingBundleCoordinate = null;
+        if (!componentToEnrichVersionToBundles.isEmpty()) {
+            // If there is only one supporting bundle, choose it, otherwise carry out additional analysis
+            if (componentToEnrichVersionToBundles.size() == 1) {
+                BundleDetails enrichingBundleDetails = componentToEnrichVersionToBundles.entrySet().iterator().next().getValue().getBundleDetails();
+                enrichingBundleCoordinate = enrichingBundleDetails.getCoordinate();
+                // Adjust the bundle to reflect the values we learned from the Extension Manager
+                componentToEnrich.setBundleInformation(enrichingBundleCoordinate);
+                componentToEnrich.setDependsUponBundleCoordinate(enrichingBundleDetails.getDependencyCoordinate());
+            } else {
+                // multiple options
+                final Set<String> componentToEnrichBundleVersions = componentToEnrichVersionToBundles.values().stream()
+                        .map(bundle -> bundle.getBundleDetails().getCoordinate().getVersion()).collect(Collectors.toSet());
+                // Select the last version of those available for the enriching bundle
+                final String bundleVersion = componentToEnrichBundleVersions.stream().sorted().reduce((version, otherVersion) -> otherVersion).get();
+                final BundleCoordinate enrichingCoordinate = componentToEnrichVersionToBundles.get(bundleVersion).getBundleDetails().getCoordinate();
+                componentToEnrich.setBundleInformation(enrichingCoordinate);
+                logger.warn("Multiple enriching bundle options were available for component {}.  The automatically selected enriching bundle was {}",
+                        new Object[]{componentToEnrich.getComponentClass(), enrichingCoordinate});
+            }
+        } else {
+            logger.warn("Could not find any eligible bundles for {}.  Automatic start of the flow cannot be guaranteed.", componentToEnrich.getComponentClass());
+        }
+    }
+
+    /**
+     * Find dependent components for the nodes provided.
+     * <p>
+     * We do not have any other information in a generic sense other than that the  properties that make use of UUIDs
+     * are eligible to be dependent components; there is no typing that a value is an ID and not just the format of a UUID.
+     * If we find a property that has a UUID as its value, we take note and create a mapping.
+     * If it is a valid ID of another component, we can use this to pair up versions, otherwise, it is ignored.
+     *
+     * @param parentNodes component nodes to map to dependent components (e.g. Processor -> Controller Service)
+     * @return a map of component IDs to their metadata about their relationship
+     */
+    protected static Map<String, EnrichingElementAdapter> mapComponents(NodeList parentNodes) {
+        final Map<String, EnrichingElementAdapter> componentReferenceMap = new HashMap<>();
+        for (int compIdx = 0; compIdx < parentNodes.getLength(); compIdx++) {
+            final Node subjComponent = parentNodes.item(compIdx);
+            final EnrichingElementAdapter enrichingElement = new EnrichingElementAdapter((Element) subjComponent);
+            componentReferenceMap.put(enrichingElement.getComponentId(), enrichingElement);
+        }
+        return componentReferenceMap;
+    }
+
+
+    /*
+     * Convenience class to aid in interacting with the XML elements pertaining to a bundle-able component
+     */
+    public static class EnrichingElementAdapter {
+        public static final String BUNDLE_ELEMENT_NAME = "bundle";
+
+        public static final String GROUP_ELEMENT_NAME = "group";
+        public static final String ARTIFACT_ELEMENT_NAME = "artifact";
+        public static final String VERSION_ELEMENT_NAME = "version";
+
+        public static final String PROPERTY_ELEMENT_NAME = "property";
+
+        // Source object
+        private Element rawElement;
+
+        // Metadata
+        private String id;
+        private String compClass;
+        private Element bundleElement;
+        private BundleCoordinate dependsUponBundleCoordinate;
+
+        public EnrichingElementAdapter(Element element) {
+            this.rawElement = element;
+        }
+
+        public String getComponentId() {
+            if (this.id == null) {
+                this.id = lookupValue("id");
+            }
+            return this.id;
+        }
+
+        public String getComponentClass() {
+            if (this.compClass == null) {
+                this.compClass = lookupValue("class");
+            }
+            return compClass;
+        }
+
+        public Element getBundleElement() {
+            if (this.bundleElement == null) {
+                // Check if the raw element has bundle information, returning it if it does
+                final NodeList bundleElements = this.rawElement.getElementsByTagName(BUNDLE_ELEMENT_NAME);
+                if (bundleElements != null && bundleElements.getLength() == 1) {
+                    this.bundleElement = (Element) bundleElements.item(0);
+                }
+            }
+            return this.bundleElement;
+        }
+
+        public List<Element> getProperties() {
+            return FlowParser.getChildrenByTagName(this.rawElement, PROPERTY_ELEMENT_NAME);
+        }
+
+        private String lookupValue(String elementName) {
+            return FlowParser.getChildrenByTagName(this.rawElement, elementName).get(0).getTextContent();
+        }
+
+        public void setBundleInformation(final BundleCoordinate bundleCoordinate) {
+            // If we are handling a component that does not yet have bundle information, create a placeholder element
+            if (this.bundleElement == null) {
+                this.bundleElement = this.rawElement.getOwnerDocument().createElement(BUNDLE_ELEMENT_NAME);
+                for (String elementTag : Arrays.asList(GROUP_ELEMENT_NAME, ARTIFACT_ELEMENT_NAME, VERSION_ELEMENT_NAME)) {
+                    this.bundleElement.appendChild(this.bundleElement.getOwnerDocument().createElement(elementTag));
+                }
+                this.rawElement.appendChild(this.bundleElement);
+            }
+            setBundleInformation(bundleCoordinate.getGroup(), bundleCoordinate.getId(), bundleCoordinate.getVersion());
+        }
+
+        private void setBundleInformation(String group, String artifact, String version) {
+            this.bundleElement.getElementsByTagName(GROUP_ELEMENT_NAME).item(0).setTextContent(group);
+            this.bundleElement.getElementsByTagName(ARTIFACT_ELEMENT_NAME).item(0).setTextContent(artifact);
+            this.bundleElement.getElementsByTagName(VERSION_ELEMENT_NAME).item(0).setTextContent(version);
+        }
+
+        public void setDependsUponBundleCoordinate(BundleCoordinate dependsUponBundleCoordinate) {
+            this.dependsUponBundleCoordinate = dependsUponBundleCoordinate;
+        }
+
+        private String getBundleElementPropertyContent(String elementName) {
+            return (getBundleElement() == null) ? null : FlowParser.getChildrenByTagName(this.bundleElement, elementName).get(0).getTextContent();
+        }
+
+        public String getBundleGroup() {
+            return getBundleElementPropertyContent(GROUP_ELEMENT_NAME);
+        }
+
+        public String getBundleId() {
+            return getBundleElementPropertyContent(ARTIFACT_ELEMENT_NAME);
+        }
+
+        public String getBundleVersion() {
+            return getBundleElementPropertyContent(VERSION_ELEMENT_NAME);
+        }
+    }
+}
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
new file mode 100644
index 0000000..0b888fc
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-headless-server/src/main/java/org/apache/nifi/headless/HeadlessNiFiServer.java
@@ -0,0 +1,236 @@
+/*
+ * 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.headless;
+
+import org.apache.nifi.NiFiServer;
+import org.apache.nifi.admin.service.AuditService;
+import org.apache.nifi.admin.service.impl.StandardAuditService;
+import org.apache.nifi.authorization.AuthorizationRequest;
+import org.apache.nifi.authorization.AuthorizationResult;
+import org.apache.nifi.authorization.Authorizer;
+import org.apache.nifi.authorization.AuthorizerConfigurationContext;
+import org.apache.nifi.authorization.AuthorizerInitializationContext;
+import org.apache.nifi.authorization.FlowParser;
+import org.apache.nifi.authorization.exception.AuthorizationAccessException;
+import org.apache.nifi.authorization.exception.AuthorizerCreationException;
+import org.apache.nifi.authorization.exception.AuthorizerDestructionException;
+import org.apache.nifi.bundle.Bundle;
+import org.apache.nifi.controller.FlowController;
+import org.apache.nifi.controller.StandardFlowService;
+import org.apache.nifi.controller.flow.FlowManager;
+import org.apache.nifi.controller.repository.FlowFileEventRepository;
+import org.apache.nifi.controller.repository.metrics.RingBufferEventRepository;
+import org.apache.nifi.diagnostics.DiagnosticsDump;
+import org.apache.nifi.diagnostics.DiagnosticsDumpElement;
+import org.apache.nifi.diagnostics.DiagnosticsFactory;
+import org.apache.nifi.diagnostics.ThreadDumpTask;
+import org.apache.nifi.diagnostics.bootstrap.BootstrapDiagnosticsFactory;
+import org.apache.nifi.encrypt.StringEncryptor;
+import org.apache.nifi.events.VolatileBulletinRepository;
+import org.apache.nifi.nar.ExtensionDiscoveringManager;
+import org.apache.nifi.nar.ExtensionManagerHolder;
+import org.apache.nifi.nar.ExtensionMapping;
+import org.apache.nifi.nar.StandardExtensionDiscoveringManager;
+import org.apache.nifi.registry.VariableRegistry;
+import org.apache.nifi.registry.flow.StandardFlowRegistryClient;
+import org.apache.nifi.registry.variable.FileBasedVariableRegistry;
+import org.apache.nifi.reporting.BulletinRepository;
+import org.apache.nifi.services.FlowService;
+import org.apache.nifi.util.NiFiProperties;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedWriter;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.util.List;
+import java.util.Set;
+
+/**
+ */
+public class HeadlessNiFiServer implements NiFiServer {
+
+    private static final Logger logger = LoggerFactory.getLogger(HeadlessNiFiServer.class);
+    private NiFiProperties props;
+    private Bundle systemBundle;
+    private Set<Bundle> bundles;
+    private FlowService flowService;
+    private DiagnosticsFactory diagnosticsFactory;
+
+    private static final String DEFAULT_SENSITIVE_PROPS_KEY = "nififtw!";
+
+    /**
+     * Default constructor
+     */
+    public HeadlessNiFiServer() {
+    }
+
+    public void start() {
+        try {
+
+            // Create a standard extension manager and discover extensions
+            final ExtensionDiscoveringManager extensionManager = new StandardExtensionDiscoveringManager();
+            extensionManager.discoverExtensions(systemBundle, bundles);
+            extensionManager.logClassLoaderMapping();
+
+            // Set the extension manager into the holder which makes it available to the Spring context via a factory bean
+            ExtensionManagerHolder.init(extensionManager);
+
+            // Enrich the flow xml using the Extension Manager mapping
+            final FlowParser flowParser = new FlowParser();
+            final FlowEnricher flowEnricher = new FlowEnricher(this, flowParser, props);
+            flowEnricher.enrichFlowWithBundleInformation();
+            logger.info("Loading Flow...");
+
+            FlowFileEventRepository flowFileEventRepository = new RingBufferEventRepository(5);
+            AuditService auditService = new StandardAuditService();
+            Authorizer authorizer = new Authorizer() {
+                @Override
+                public AuthorizationResult authorize(AuthorizationRequest request) throws AuthorizationAccessException {
+                    return AuthorizationResult.approved();
+                }
+
+                @Override
+                public void initialize(AuthorizerInitializationContext initializationContext) throws AuthorizerCreationException {
+                    // do nothing
+                }
+
+                @Override
+                public void onConfigured(AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException {
+                    // do nothing
+                }
+
+                @Override
+                public void preDestruction() throws AuthorizerDestructionException {
+                    // do nothing
+                }
+            };
+
+            final String sensitivePropAlgorithmVal = props.getProperty(StringEncryptor.NF_SENSITIVE_PROPS_ALGORITHM);
+            final String sensitivePropProviderVal = props.getProperty(StringEncryptor.NF_SENSITIVE_PROPS_PROVIDER);
+            final String sensitivePropValueNifiPropVar = props.getProperty(StringEncryptor.NF_SENSITIVE_PROPS_KEY, DEFAULT_SENSITIVE_PROPS_KEY);
+
+            StringEncryptor encryptor = StringEncryptor.createEncryptor(sensitivePropAlgorithmVal, sensitivePropProviderVal, sensitivePropValueNifiPropVar);
+            VariableRegistry variableRegistry = new FileBasedVariableRegistry(props.getVariableRegistryPropertiesPaths());
+            BulletinRepository bulletinRepository = new VolatileBulletinRepository();
+            StandardFlowRegistryClient flowRegistryClient = new StandardFlowRegistryClient();
+            flowRegistryClient.setProperties(props);
+
+            FlowController flowController = FlowController.createStandaloneInstance(
+                    flowFileEventRepository,
+                    props,
+                    authorizer,
+                    auditService,
+                    encryptor,
+                    bulletinRepository,
+                    variableRegistry,
+                    flowRegistryClient,
+                    extensionManager
+                    );
+
+            flowService = StandardFlowService.createStandaloneInstance(
+                    flowController,
+                    props,
+                    encryptor,
+                    null, // revision manager
+                    authorizer);
+
+            diagnosticsFactory = new BootstrapDiagnosticsFactory();
+            ((BootstrapDiagnosticsFactory) diagnosticsFactory).setFlowController(flowController);
+            ((BootstrapDiagnosticsFactory) diagnosticsFactory).setNifiProperties(props);
+
+            // start and load the flow
+            flowService.start();
+            flowService.load(null);
+            flowController.onFlowInitialized(true);
+            FlowManager flowManager = flowController.getFlowManager();
+            flowManager.getGroup(flowManager.getRootGroupId()).startProcessing();
+
+            logger.info("Flow loaded successfully.");
+        } catch (Exception e) {
+            // ensure the flow service is terminated
+            if (flowService != null && flowService.isRunning()) {
+                flowService.stop(false);
+            }
+            startUpFailure(new Exception("Unable to load flow due to: " + e, e));
+        }
+    }
+
+    private void startUpFailure(Throwable t) {
+        System.err.println("Failed to start flow service: " + t.getMessage());
+        System.err.println("Shutting down...");
+        logger.warn("Failed to start headless server... shutting down.", t);
+        System.exit(1);
+    }
+
+    @Override
+    public void initialize(NiFiProperties properties, Bundle systemBundle, Set<Bundle> bundles, ExtensionMapping extensionMapping) {
+        this.props = properties;
+        this.systemBundle = systemBundle;
+        this.bundles = bundles;
+    }
+
+    @Override
+    public DiagnosticsFactory getDiagnosticsFactory() {
+        // The diagnosticsFactory is initialized during server startup. If the diagnostics factory happens to be
+        // requested before the Server starts, or after the server fails to start, we cannot provide the fully initialized
+        // diagnostics factory. But it is still helpful to provide what we can, so we will provide the Thread Dump Factory.
+        return diagnosticsFactory == null ? getThreadDumpFactory() : diagnosticsFactory;
+    }
+
+    @Override
+    public DiagnosticsFactory getThreadDumpFactory() {
+        return new ThreadDumpDiagnosticsFactory();
+    }
+
+    public void stop() {
+        try {
+            flowService.stop(false);
+        } catch (Exception e) {
+            String msg = "Problem occurred ensuring flow controller or repository was properly terminated due to " + e;
+            if (logger.isDebugEnabled()) {
+                logger.warn(msg, e);
+            } else {
+                logger.warn(msg);
+            }
+        }
+    }
+
+    protected List<Bundle> getBundles(final String bundleClass) {
+        return ExtensionManagerHolder.getExtensionManager().getBundles(bundleClass);
+    }
+
+    private static class ThreadDumpDiagnosticsFactory implements DiagnosticsFactory {
+        @Override
+        public DiagnosticsDump create(final boolean verbose) {
+            return new DiagnosticsDump() {
+                @Override
+                public void writeTo(final OutputStream out) throws IOException {
+                    final DiagnosticsDumpElement threadDumpElement = new ThreadDumpTask().captureDump(verbose);
+                    final BufferedWriter writer = new BufferedWriter(new OutputStreamWriter(out));
+                    for (final String detail : threadDumpElement.getDetails()) {
+                        writer.write(detail);
+                        writer.write("\n");
+                    }
+
+                    writer.flush();
+                }
+            };
+        }
+    }
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-headless-server/src/main/resources/META-INF/services/org.apache.nifi.NiFiServer b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-headless-server/src/main/resources/META-INF/services/org.apache.nifi.NiFiServer
new file mode 100644
index 0000000..f317186
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-headless-server/src/main/resources/META-INF/services/org.apache.nifi.NiFiServer
@@ -0,0 +1,15 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+org.apache.nifi.headless.HeadlessNiFiServer
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/pom.xml
index 3aaa2d4..d181938 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/pom.xml
@@ -34,6 +34,10 @@
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-framework-api</artifactId>
         </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-server-api</artifactId>
+        </dependency>
     </dependencies>
     <build>
         <plugins>
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarClassLoaders.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarClassLoaders.java
index 380ade0..d830253 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarClassLoaders.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarClassLoaders.java
@@ -16,6 +16,7 @@
  */
 package org.apache.nifi.nar;
 
+import org.apache.nifi.NiFiServer;
 import org.apache.nifi.bundle.Bundle;
 import org.apache.nifi.bundle.BundleCoordinate;
 import org.apache.nifi.bundle.BundleDetails;
@@ -36,7 +37,9 @@ import java.util.List;
 import java.util.Map;
 import java.util.Objects;
 import java.util.Optional;
+import java.util.ServiceLoader;
 import java.util.Set;
+import java.util.stream.Collectors;
 
 
 /**
@@ -58,6 +61,7 @@ public final class NarClassLoaders {
         private final File extensionWorkingDir;
         private final Bundle frameworkBundle;
         private final Bundle jettyBundle;
+        private final NiFiServer serverInstance;
         private final Map<String, Bundle> bundles;
 
         private InitContext(
@@ -65,11 +69,13 @@ public final class NarClassLoaders {
                 final File extensionDir,
                 final Bundle frameworkBundle,
                 final Bundle jettyBundle,
+                final NiFiServer serverInstance,
                 final Map<String, Bundle> bundles) {
             this.frameworkWorkingDir = frameworkDir;
             this.extensionWorkingDir = extensionDir;
             this.frameworkBundle = frameworkBundle;
             this.jettyBundle = jettyBundle;
+            this.serverInstance = serverInstance;
             this.bundles = bundles;
         }
     }
@@ -158,6 +164,7 @@ public final class NarClassLoaders {
             narWorkingDirContents.addAll(Arrays.asList(extensionsWorkingDirContents));
         }
 
+        NiFiServer serverInstance = null;
         if (!narWorkingDirContents.isEmpty()) {
             final List<BundleDetails> narDetails = new ArrayList<>();
             final Map<String,String> narCoordinatesToWorkingDir = new HashMap<>();
@@ -206,11 +213,8 @@ public final class NarClassLoaders {
                 narIdBundleLookup.computeIfAbsent(narDetail.getCoordinate().getId(), id -> new HashSet<>()).add(narDetail.getCoordinate());
             }
 
-            // ensure the jetty nar was found
-            if (jettyClassLoader == null) {
-                throw new IllegalStateException("Unable to locate Jetty bundle.");
-            }
-
+            // Keep track of NiFiServer implementations
+            Map<NiFiServer, String> niFiServers = new HashMap<>();
             int narCount;
             do {
                 // record the number of nars to be loaded
@@ -260,14 +264,32 @@ public final class NarClassLoaders {
                     final ClassLoader bundleClassLoader = narClassLoader;
                     if (bundleClassLoader != null) {
                         narDirectoryBundleLookup.put(narDetail.getWorkingDirectory().getCanonicalPath(), new Bundle(narDetail, bundleClassLoader));
-                        narCoordinateClassLoaderLookup.put(narDetail.getCoordinate().getCoordinate(), narClassLoader);
+                        String coordinate = narDetail.getCoordinate().getCoordinate();
+                        narCoordinateClassLoaderLookup.put(coordinate, narClassLoader);
                         narDetailsIter.remove();
+                        // Search for a NiFiServer implementation
+                        ServiceLoader<NiFiServer> niFiServerServiceLoader = ServiceLoader.load(NiFiServer.class, narClassLoader);
+                        for (NiFiServer server : niFiServerServiceLoader) {
+                            niFiServers.put(server, coordinate);
+                        }
                     }
                 }
-
                 // attempt to load more if some were successfully loaded this iteration
             } while (narCount != narDetails.size());
 
+            // Ensure exactly one NiFiServer implementation, otherwise report none or multiples found
+            if (niFiServers.size() == 0) {
+                throw new IOException("No implementations of NiFiServer found, there must be exactly one implementation.");
+            } else if (niFiServers.size() > 1) {
+                String sb = "Expected exactly one implementation of NiFiServer but found " + niFiServers.size() + ": " +
+                        niFiServers.entrySet().stream().map((entry) -> entry.getKey().getClass().getName() + " from " + entry.getValue()).collect(Collectors.joining(", "));
+                throw new IOException(sb);
+            } else {
+                Map.Entry<NiFiServer, String> nifiServer = niFiServers.entrySet().iterator().next();
+                serverInstance = nifiServer.getKey();
+                logger.info("Found NiFiServer implementation {} in {}", new Object[]{serverInstance.getClass().getName(), nifiServer.getValue()});
+            }
+
             // see if any nars couldn't be loaded
             for (final BundleDetails narDetail : narDetails) {
                 logger.warn(String.format("Unable to resolve required dependency '%s'. Skipping NAR '%s'",
@@ -285,11 +307,7 @@ public final class NarClassLoaders {
                 .filter(b -> b.getBundleDetails().getCoordinate().getId().equals(JETTY_NAR_ID))
                 .findFirst().orElse(null);
 
-        if (jettyBundle == null) {
-            throw new IllegalStateException("Unable to locate Jetty bundle.");
-        }
-
-        return new InitContext(frameworkWorkingDir, extensionsWorkingDir, frameworkBundle, jettyBundle, new LinkedHashMap<>(narDirectoryBundleLookup));
+        return new InitContext(frameworkWorkingDir, extensionsWorkingDir, frameworkBundle, jettyBundle, serverInstance, new LinkedHashMap<>(narDirectoryBundleLookup));
     }
 
     /**
@@ -369,8 +387,15 @@ public final class NarClassLoaders {
 
         final BundleCoordinate bundleDependencyCoordinate = bundleDetail.getDependencyCoordinate();
         if (bundleDependencyCoordinate == null) {
-            final ClassLoader jettyClassLoader = getJettyBundle().getClassLoader();
-            bundleClassLoader = createNarClassLoader(bundleDetail.getWorkingDirectory(), jettyClassLoader);
+            final ClassLoader parentClassLoader;
+            Bundle jettyBundle = getJettyBundle();
+            if (jettyBundle != null) {
+                parentClassLoader = jettyBundle.getClassLoader();
+            } else {
+                // If there is no Jetty bundle, assume to be "headless"
+                parentClassLoader = null;
+            }
+            bundleClassLoader = createNarClassLoader(bundleDetail.getWorkingDirectory(), parentClassLoader);
         } else {
             final Optional<Bundle> dependencyBundle = getBundle(bundleDependencyCoordinate);
 
@@ -423,7 +448,7 @@ public final class NarClassLoaders {
 
                     logger.error("Unable to load NAR with coordinates {} and working directory {} " +
                                     "because another NAR with the same coordinates already exists at {}",
-                            new Object[]{unpackedNarCoordinate, unpackedNarWorkingDir, existingNarWorkingDir});
+                            unpackedNarCoordinate, unpackedNarWorkingDir, existingNarWorkingDir);
                 } else {
                     narDetails.add(narDetail);
                 }
@@ -502,6 +527,19 @@ public final class NarClassLoaders {
     }
 
     /**
+     * @return the Server class Bundle (NiFi Web/UI or MiNiFi)
+     *
+     * @throws IllegalStateException if the server Bundle has not been loaded
+     */
+    public NiFiServer getServer() {
+        if (initContext == null) {
+            throw new IllegalStateException("Server bundle has not been loaded.");
+        }
+
+        return initContext.serverInstance;
+    }
+
+    /**
      * @param extensionWorkingDirectory the directory
      * @return the bundle for the specified working directory. Returns
      * null when no bundle exists for the specified working directory
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 bb87ec3..9fa92bb 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,8 +24,6 @@ import org.apache.nifi.util.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import static java.lang.String.format;
-
 import java.io.BufferedReader;
 import java.io.File;
 import java.io.FileFilter;
@@ -53,6 +51,8 @@ import java.util.jar.JarEntry;
 import java.util.jar.JarFile;
 import java.util.jar.Manifest;
 
+import static java.lang.String.format;
+
 /**
  *
  */
@@ -243,7 +243,10 @@ public final class NarUnpacker {
 
     public static void mapExtension(final File unpackedNar, final BundleCoordinate bundleCoordinate, final File docsDirectory, final ExtensionMapping mapping) throws IOException {
         final File bundledDependencies = new File(unpackedNar, BUNDLED_DEPENDENCIES_DIRECTORY);
-        unpackBundleDocs(docsDirectory, mapping, bundleCoordinate, bundledDependencies);
+        // If docsDirectory is null, assume NiFi is "headless" (no UI or REST API) and thus no docs are to be generated
+        if (docsDirectory != null) {
+            unpackBundleDocs(docsDirectory, mapping, bundleCoordinate, bundledDependencies);
+        }
     }
 
     private static void unpackBundleDocs(final File docsDirectory, final ExtensionMapping mapping, final BundleCoordinate bundleCoordinate, final File bundledDirectory) throws IOException {
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/pom.xml
index 392783d..6009eeb 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/pom.xml
@@ -29,6 +29,11 @@
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-server-api</artifactId>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-nar-utils</artifactId>
             <scope>compile</scope>
         </dependency>
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/main/java/org/apache/nifi/BootstrapListener.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/main/java/org/apache/nifi/BootstrapListener.java
index c0dbcbc..56394d0 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/main/java/org/apache/nifi/BootstrapListener.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/main/java/org/apache/nifi/BootstrapListener.java
@@ -40,14 +40,14 @@ public class BootstrapListener {
 
     private static final Logger logger = LoggerFactory.getLogger(BootstrapListener.class);
 
-    private final NiFi nifi;
+    private final NiFiEntryPoint nifi;
     private final int bootstrapPort;
     private final String secretKey;
 
     private volatile Listener listener;
     private volatile ServerSocket serverSocket;
 
-    public BootstrapListener(final NiFi nifi, final int bootstrapPort) {
+    public BootstrapListener(final NiFiEntryPoint nifi, final int bootstrapPort) {
         this.nifi = nifi;
         this.bootstrapPort = bootstrapPort;
         secretKey = UUID.randomUUID().toString();
@@ -73,6 +73,13 @@ public class BootstrapListener {
         sendCommand("PORT", new String[] { String.valueOf(localPort), secretKey});
     }
 
+    public void reload() throws IOException {
+        if (listener != null) {
+            listener.stop();
+        }
+        sendCommand("RELOAD", new String[]{});
+    }
+
     public void stop() {
         if (listener != null) {
             listener.stop();
@@ -176,11 +183,16 @@ public class BootstrapListener {
                                         echoPing(socket.getOutputStream());
                                         logger.debug("Responded to PING request from Bootstrap");
                                         break;
+                                    case RELOAD:
+                                        logger.info("Received RELOAD request from Bootstrap");
+                                        echoReload(socket.getOutputStream());
+                                        nifi.shutdownHook(true);
+                                        return;
                                     case SHUTDOWN:
                                         logger.info("Received SHUTDOWN request from Bootstrap");
                                         echoShutdown(socket.getOutputStream());
                                         socket.close();
-                                        nifi.shutdownHook();
+                                        nifi.shutdownHook(false);
                                         return;
                                     case DUMP:
                                         logger.info("Received DUMP request from Bootstrap");
@@ -242,6 +254,11 @@ public class BootstrapListener {
         out.flush();
     }
 
+    private void echoReload(final OutputStream out) throws IOException {
+        out.write("RELOAD\n".getBytes(StandardCharsets.UTF_8));
+        out.flush();
+    }
+
     @SuppressWarnings("resource")  // we don't want to close the stream, as the caller will do that
     private BootstrapRequest readRequest(final InputStream in) throws IOException {
         // We want to ensure that we don't try to read data from an InputStream directly
@@ -282,6 +299,7 @@ public class BootstrapListener {
 
     private static class BootstrapRequest {
         public enum RequestType {
+            RELOAD,
             SHUTDOWN,
             DUMP,
             DIAGNOSTICS,
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 acfed37..96199c1 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
@@ -32,7 +32,6 @@ import java.io.File;
 import java.io.FileWriter;
 import java.io.IOException;
 import java.lang.Thread.UncaughtExceptionHandler;
-import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
 import java.net.MalformedURLException;
@@ -44,8 +43,8 @@ import java.nio.file.Paths;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
-import java.util.Set;
 import java.util.Random;
+import java.util.Set;
 import java.util.Timer;
 import java.util.TimerTask;
 import java.util.concurrent.Executors;
@@ -56,7 +55,7 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 
-public class NiFi {
+public class NiFi implements NiFiEntryPoint {
 
     private static final Logger LOGGER = LoggerFactory.getLogger(NiFi.class);
     private static final String KEY_FILE_FLAG = "-K";
@@ -142,15 +141,17 @@ public class NiFi {
 
         final Set<Bundle> narBundles = narClassLoaders.getBundles();
 
-        // load the server from the framework classloader
-        Thread.currentThread().setContextClassLoader(frameworkClassLoader);
-        Class<?> jettyServer = Class.forName("org.apache.nifi.web.server.JettyServer", true, frameworkClassLoader);
-        Constructor<?> jettyConstructor = jettyServer.getConstructor(NiFiProperties.class, Set.class);
-
         final long startTime = System.nanoTime();
-        nifiServer = (NiFiServer) jettyConstructor.newInstance(properties, narBundles);
-        nifiServer.setExtensionMapping(extensionMapping);
-        nifiServer.setBundles(systemBundle, narBundles);
+        nifiServer = narClassLoaders.getServer();
+        if (nifiServer == null) {
+            throw new IllegalStateException("Unable to find a NiFiServer implementation.");
+        }
+        Thread.currentThread().setContextClassLoader(nifiServer.getClass().getClassLoader());
+        // Filter out the framework NAR from being loaded by the NiFiServer
+        nifiServer.initialize(properties,
+                systemBundle,
+                narBundles,
+                extensionMapping);
 
         if (shutdown) {
             LOGGER.info("NiFi has been shutdown via NiFi Bootstrap. Will not start Controller");
@@ -167,7 +168,7 @@ public class NiFi {
         }
     }
 
-    NiFiServer getServer() {
+    public NiFiServer getServer() {
         return nifiServer;
     }
 
@@ -186,7 +187,7 @@ public class NiFi {
             @Override
             public void run() {
                 // shutdown the jetty server
-                shutdownHook();
+                shutdownHook(false);
             }
         }));
     }
@@ -214,7 +215,7 @@ public class NiFi {
         return new URLClassLoader(urls.toArray(new URL[0]), Thread.currentThread().getContextClassLoader());
     }
 
-    protected void shutdownHook() {
+    public void shutdownHook(boolean isReload) {
         try {
             shutdown();
         } catch (final Throwable t) {
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/main/java/org/apache/nifi/NiFiServer.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/main/java/org/apache/nifi/NiFiEntryPoint.java
similarity index 64%
copy from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/main/java/org/apache/nifi/NiFiServer.java
copy to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/main/java/org/apache/nifi/NiFiEntryPoint.java
index 2169579..3867333 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/main/java/org/apache/nifi/NiFiServer.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/main/java/org/apache/nifi/NiFiEntryPoint.java
@@ -16,26 +16,9 @@
  */
 package org.apache.nifi;
 
-import org.apache.nifi.bundle.Bundle;
-import org.apache.nifi.diagnostics.DiagnosticsFactory;
-import org.apache.nifi.nar.ExtensionMapping;
+public interface NiFiEntryPoint {
 
-import java.util.Set;
+    NiFiServer getServer();
 
-/**
- *
- */
-public interface NiFiServer {
-
-    void start();
-
-    void setExtensionMapping(ExtensionMapping extensionMapping);
-
-    void setBundles(Bundle systemBundle, Set<Bundle> bundles);
-
-    void stop();
-
-    DiagnosticsFactory getDiagnosticsFactory();
-
-    DiagnosticsFactory getThreadDumpFactory();
+    void shutdownHook(boolean isReload);
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/main/java/org/apache/nifi/NiFiServer.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/main/java/org/apache/nifi/headless/FlowEnrichmentException.java
similarity index 52%
copy from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/main/java/org/apache/nifi/NiFiServer.java
copy to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/main/java/org/apache/nifi/headless/FlowEnrichmentException.java
index 2169579..acbe91b 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/main/java/org/apache/nifi/NiFiServer.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/main/java/org/apache/nifi/headless/FlowEnrichmentException.java
@@ -1,41 +1,39 @@
-/*
+/**
  * 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
- *
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
  * 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;
-
-import org.apache.nifi.bundle.Bundle;
-import org.apache.nifi.diagnostics.DiagnosticsFactory;
-import org.apache.nifi.nar.ExtensionMapping;
-
-import java.util.Set;
-
-/**
- *
- */
-public interface NiFiServer {
-
-    void start();
+package org.apache.nifi.headless;
 
-    void setExtensionMapping(ExtensionMapping extensionMapping);
+public class FlowEnrichmentException extends Exception {
+    public FlowEnrichmentException() {
+        super();
+    }
 
-    void setBundles(Bundle systemBundle, Set<Bundle> bundles);
+    public FlowEnrichmentException(String message) {
+        super(message);
+    }
 
-    void stop();
+    public FlowEnrichmentException(String message, Throwable cause) {
+        super(message, cause);
+    }
 
-    DiagnosticsFactory getDiagnosticsFactory();
+    public FlowEnrichmentException(Throwable cause) {
+        super(cause);
+    }
 
-    DiagnosticsFactory getThreadDumpFactory();
+    protected FlowEnrichmentException(String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) {
+        super(message, cause, enableSuppression, writableStackTrace);
+    }
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/pom.xml
index 948ca89..9c0c971 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/pom.xml
@@ -30,86 +30,102 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-api</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-server-api</artifactId>
             <scope>compile</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-runtime</artifactId>
-            <scope>compile</scope>
+            <scope>provided</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-framework-nar-loading-utils</artifactId>
-            <scope>compile</scope>
+            <scope>provided</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-nar-utils</artifactId>
-            <scope>compile</scope>
+            <scope>provided</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-properties</artifactId>
-            <scope>compile</scope>
+            <scope>provided</scope>
         </dependency>
         <dependency>
             <groupId>org.eclipse.jetty</groupId>
             <artifactId>jetty-server</artifactId>
+            <scope>provided</scope>
         </dependency>
         <dependency>
             <groupId>org.eclipse.jetty</groupId>
             <artifactId>jetty-servlet</artifactId>
+            <scope>provided</scope>
         </dependency>
         <dependency>
             <groupId>org.eclipse.jetty</groupId>
             <artifactId>jetty-webapp</artifactId>
+            <scope>provided</scope>
         </dependency>
         <dependency>
             <groupId>org.eclipse.jetty</groupId>
             <artifactId>jetty-deploy</artifactId>
+            <scope>provided</scope>
         </dependency>
         <dependency>
             <groupId>org.eclipse.jetty</groupId>
             <artifactId>jetty-annotations</artifactId>
+            <scope>provided</scope>
         </dependency>
         <dependency>
             <groupId>org.eclipse.jetty</groupId>
             <artifactId>jetty-servlets</artifactId>
+            <scope>provided</scope>
         </dependency>
         <dependency>
             <groupId>javax.ws.rs</groupId>
             <artifactId>javax.ws.rs-api</artifactId>
+            <scope>provided</scope>
         </dependency>
         <dependency>
             <groupId>org.glassfish.jersey.core</groupId>
             <artifactId>jersey-client</artifactId>
+            <scope>provided</scope>
         </dependency>
         <dependency>
             <groupId>org.glassfish.jersey.media</groupId>
             <artifactId>jersey-media-json-jackson</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.glassfish.jersey.media</groupId>
-            <artifactId>jersey-media-multipart</artifactId>
+            <scope>provided</scope>
         </dependency>
         <dependency>
             <groupId>org.glassfish.jersey.ext</groupId>
             <artifactId>jersey-spring4</artifactId>
         </dependency>
         <dependency>
+            <groupId>org.glassfish.jersey.core</groupId>
+            <artifactId>jersey-server</artifactId>
+            <version>${jersey.version}</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
             <groupId>org.aspectj</groupId>
             <artifactId>aspectjweaver</artifactId>
-            <scope>compile</scope>
+            <scope>provided</scope>
         </dependency>
         <dependency>
             <groupId>cglib</groupId> 
             <artifactId>cglib-nodep</artifactId> 
-            <scope>compile</scope>
+            <scope>provided</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-client-dto</artifactId>
-            <scope>compile</scope>
+            <scope>provided</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
@@ -129,7 +145,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-web-security</artifactId>
-            <scope>compile</scope>
+            <scope>provided</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
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 6fc9af9..0fde717 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
@@ -132,16 +132,13 @@ public class JettyServer implements NiFiServer, ExtensionUiLoader {
     private static final String CONTAINER_INCLUDE_PATTERN_KEY = "org.eclipse.jetty.server.webapp.ContainerIncludeJarPattern";
     private static final String CONTAINER_INCLUDE_PATTERN_VALUE = ".*/[^/]*servlet-api-[^/]*\\.jar$|.*/javax.servlet.jsp.jstl-.*\\\\.jar$|.*/[^/]*taglibs.*\\.jar$";
 
-    private static final FileFilter WAR_FILTER = new FileFilter() {
-        @Override
-        public boolean accept(File pathname) {
-            final String nameToTest = pathname.getName().toLowerCase();
-            return nameToTest.endsWith(".war") && pathname.isFile();
-        }
+    private static final FileFilter WAR_FILTER = pathname -> {
+        final String nameToTest = pathname.getName().toLowerCase();
+        return nameToTest.endsWith(".war") && pathname.isFile();
     };
 
-    private final Server server;
-    private final NiFiProperties props;
+    private Server server;
+    private NiFiProperties props;
 
     private Bundle systemBundle;
     private Set<Bundle> bundles;
@@ -162,13 +159,18 @@ public class JettyServer implements NiFiServer, ExtensionUiLoader {
 
     private DeploymentManager deploymentManager;
 
-    public JettyServer(final NiFiProperties props, final Set<Bundle> bundles) {
+    /**
+     * Default no-arg constructor for ServiceLoader
+     */
+    public JettyServer() {
+    }
+
+    public void init() {
         final QueuedThreadPool threadPool = new QueuedThreadPool(props.getWebThreads());
         threadPool.setName("NiFi Web Server");
 
         // create the server
         this.server = new Server(threadPool);
-        this.props = props;
 
         // enable the annotation based configuration to ensure the jsp container is initialized properly
         final Configuration.ClassList classlist = Configuration.ClassList.setServerDefault(server);
@@ -266,7 +268,7 @@ public class JettyServer implements NiFiServer, ExtensionUiLoader {
 
         final HandlerCollection webAppContextHandlers = new HandlerCollection();
         final Collection<WebAppContext> extensionUiContexts = extensionUiInfo.getWebAppContexts();
-        extensionUiContexts.stream().forEach(c -> webAppContextHandlers.addHandler(c));
+        extensionUiContexts.forEach(webAppContextHandlers::addHandler);
 
         final ClassLoader frameworkClassLoader = getClass().getClassLoader();
 
@@ -1255,14 +1257,13 @@ public class JettyServer implements NiFiServer, ExtensionUiLoader {
     }
 
     @Override
-    public void setExtensionMapping(ExtensionMapping extensionMapping) {
-        this.extensionMapping = extensionMapping;
-    }
-
-    @Override
-    public void setBundles(Bundle systemBundle, Set<Bundle> bundles) {
+    public void initialize(NiFiProperties properties, Bundle systemBundle, Set<Bundle> bundles, ExtensionMapping extensionMapping) {
+        this.props = properties;
         this.systemBundle = systemBundle;
         this.bundles = bundles;
+        this.extensionMapping = extensionMapping;
+
+        init();
     }
 
     @Override
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/main/resources/META-INF/services/org.apache.nifi.NiFiServer b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/main/resources/META-INF/services/org.apache.nifi.NiFiServer
new file mode 100644
index 0000000..d2a3e14
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/main/resources/META-INF/services/org.apache.nifi.NiFiServer
@@ -0,0 +1,15 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+org.apache.nifi.web.server.JettyServer
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/test/groovy/org/apache/nifi/web/server/JettyServerGroovyTest.groovy b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/test/groovy/org/apache/nifi/web/server/JettyServerGroovyTest.groovy
index 23f6158..d90a758 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/test/groovy/org/apache/nifi/web/server/JettyServerGroovyTest.groovy
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/test/groovy/org/apache/nifi/web/server/JettyServerGroovyTest.groovy
@@ -239,8 +239,9 @@ class JettyServerGroovyTest extends GroovyTestCase {
         })
 
         // Act
-        JettyServer jettyServer = new JettyServer(mockProps, [] as Set<Bundle>)
-
+        JettyServer jettyServer = new JettyServer()
+        jettyServer.initialize(mockProps, null, [] as Set<Bundle>, null)
+        
         // Assert
 
         // Assertions defined above
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/pom.xml
index 8d6e5c6..51b665f 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/pom.xml
@@ -192,6 +192,11 @@
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-server-api</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-nar-utils</artifactId>
         </dependency>
         <dependency>
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-docs/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-docs/pom.xml
index e6f25f0..149dae0 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-docs/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-docs/pom.xml
@@ -48,6 +48,11 @@
             <scope>provided</scope>
         </dependency>
         <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-server-api</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
             <groupId>javax.servlet</groupId>
             <artifactId>javax.servlet-api</artifactId>
         </dependency>
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/pom.xml
index 3e5c56e..1f6bce9 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/pom.xml
@@ -48,6 +48,7 @@
         <module>nifi-mock-authorizer</module>
         <module>nifi-shell-authorizer</module>
         <module>nifi-stateless</module>
+        <module>nifi-headless-server</module>
     </modules>
     <dependencies>
         <dependency>
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework-nar/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-headless-server-nar/pom.xml
similarity index 72%
copy from nifi-nar-bundles/nifi-framework-bundle/nifi-framework-nar/pom.xml
copy to nifi-nar-bundles/nifi-framework-bundle/nifi-headless-server-nar/pom.xml
index 142cf18..04ef681 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-headless-server-nar/pom.xml
@@ -19,9 +19,9 @@
         <artifactId>nifi-framework-bundle</artifactId>
         <version>1.13.0-SNAPSHOT</version>
     </parent>
-    <artifactId>nifi-framework-nar</artifactId>
+    <artifactId>nifi-headless-server-nar</artifactId>
     <packaging>nar</packaging>
-    <description>NiFi: Framework Nar</description>
+    <description>Headless server NAR</description>
     <properties>
         <maven.javadoc.skip>true</maven.javadoc.skip>
         <source.skip>true</source.skip>
@@ -29,35 +29,49 @@
     <dependencies>
         <dependency>
             <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-jetty</artifactId>
+            <artifactId>nifi-framework-nar</artifactId>
+            <version>1.13.0-SNAPSHOT</version>
+            <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-file-authorizer</artifactId>
+            <artifactId>nifi-headless-server</artifactId>
+            <version>1.13.0-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-shell-authorizer</artifactId>
+            <artifactId>nifi-framework-core</artifactId>
+            <scope>provided</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-authorizer</artifactId>
+            <artifactId>nifi-framework-core-api</artifactId>
+            <scope>provided</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-standard-prioritizers</artifactId>
-            <version>1.13.0-SNAPSHOT</version>
+            <artifactId>nifi-framework-nar-utils</artifactId>
+            <scope>provided</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-stateless</artifactId>
-            <version>1.13.0-SNAPSHOT</version>
+            <artifactId>nifi-framework-authorization</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-framework-cluster</artifactId>
+            <scope>provided</scope>
         </dependency>
-        <!-- Override scope to compile since framework NAR won't get this from a parent NAR -->
         <dependency>
             <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-security-utils-api</artifactId>
-            <scope>compile</scope>
+            <artifactId>nifi-framework-cluster-protocol</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-web-utils</artifactId>
+            <scope>provided</scope>
         </dependency>
 
         <!-- mark these nifi artifacts as provided since it is included in the lib -->
@@ -81,6 +95,5 @@
             <artifactId>nifi-properties</artifactId>
             <scope>provided</scope>
         </dependency>
-
     </dependencies>
 </project>
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-headless-server-nar/src/main/resources/META-INF/LICENSE b/nifi-nar-bundles/nifi-framework-bundle/nifi-headless-server-nar/src/main/resources/META-INF/LICENSE
new file mode 100644
index 0000000..cc6a8e2
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-headless-server-nar/src/main/resources/META-INF/LICENSE
@@ -0,0 +1,313 @@
+
+                                 Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+   1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+   2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+   3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+   4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+   6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+   7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+   8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+   9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+
+   END OF TERMS AND CONDITIONS
+
+   APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "[]"
+      replaced with your own identifying information. (Don't include
+      the brackets!)  The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+   Copyright [yyyy] [name of copyright owner]
+
+   Licensed 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.
+
+APACHE NIFI SUBCOMPONENTS:
+
+The Apache NiFi project contains subcomponents with separate copyright
+notices and license terms. Your use of the source code for the these
+subcomponents is subject to the terms and conditions of the following
+licenses. 
+
+  The binary distribution of this product bundles 'Antlr 3' which is available
+  under a "3-clause BSD" license.  For details see http://www.antlr3.org/license.html
+
+    Copyright (c) 2010 Terence Parr
+    All rights reserved.
+
+    Redistribution and use in source and binary forms, with or without 
+    modification, are permitted provided that the following conditions are met:
+
+    Redistributions of source code must retain the above copyright notice, this 
+    list of conditions and the following disclaimer.
+    Redistributions in binary form must reproduce the above copyright notice, 
+    this list of conditions and the following disclaimer in the documentation 
+    and/or other materials provided with the distribution.
+    Neither the name of the author nor the names of its contributors may be used 
+    to endorse or promote products derived from this software without specific 
+    prior written permission.
+    THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" 
+    AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE 
+    IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE 
+    ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE 
+    LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR 
+    CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF 
+    SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS 
+    INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN 
+    CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) 
+    ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF 
+    THE POSSIBILITY OF SUCH DAMAGE.
+
+  The binary distribution of this product bundles 'Bouncy Castle JDK 1.5'
+  under an MIT style license.
+
+    Copyright (c) 2000 - 2015 The Legion of the Bouncy Castle Inc. (http://www.bouncycastle.org)
+
+    Permission is hereby granted, free of charge, to any person obtaining a copy
+    of this software and associated documentation files (the "Software"), to deal
+    in the Software without restriction, including without limitation the rights
+    to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+    copies of the Software, and to permit persons to whom the Software is
+    furnished to do so, subject to the following conditions:
+
+    The above copyright notice and this permission notice shall be included in
+    all copies or substantial portions of the Software.
+
+    THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+    IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+    FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+    AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+    LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+    OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
+    THE SOFTWARE.
+
+This product bundles 'jsonlint' which is available under an MIT license.
+
+    Copyright (C) 2012 Zachary Carter
+
+    Permission is hereby granted, free of charge, to any person obtaining a copy
+    of this software and associated documentation files (the "Software"), to deal
+    in the Software without restriction, including without limitation the rights
+    to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+    copies of the Software, and to permit persons to whom the Software is
+    furnished to do so, subject to the following conditions:
+
+    The above copyright notice and this permission notice shall be included in
+    all copies or substantial portions of the Software.
+
+    THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+    IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+    FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+    AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+    LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+    OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
+    THE SOFTWARE.
+
+This product bundles 'asm' which is available under a 3-Clause BSD style license.
+For details see http://asm.ow2.org/asmdex-license.html
+
+    Copyright (c) 2012 France Télécom
+    All rights reserved.
+
+    Redistribution and use in source and binary forms, with or without
+    modification, are permitted provided that the following conditions
+    are met:
+    1. Redistributions of source code must retain the above copyright
+       notice, this list of conditions and the following disclaimer.
+    2. Redistributions in binary form must reproduce the above copyright
+       notice, this list of conditions and the following disclaimer in the
+       documentation and/or other materials provided with the distribution.
+    3. Neither the name of the copyright holders nor the names of its
+       contributors may be used to endorse or promote products derived from
+       this software without specific prior written permission.
+
+    THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
+    AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
+    IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
+    ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE
+    LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
+    CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
+    SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
+    INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
+    CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
+    ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF
+    THE POSSIBILITY OF SUCH DAMAGE.
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-headless-server-nar/src/main/resources/META-INF/NOTICE b/nifi-nar-bundles/nifi-framework-bundle/nifi-headless-server-nar/src/main/resources/META-INF/NOTICE
new file mode 100644
index 0000000..38f3bde
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-headless-server-nar/src/main/resources/META-INF/NOTICE
@@ -0,0 +1,234 @@
+nifi-headless-server-nar
+Copyright 2014-2020 The Apache Software Foundation
+
+This product includes software developed at
+The Apache Software Foundation (http://www.apache.org/).
+
+******************
+Apache Software License v2
+******************
+
+The following binary components are provided under the Apache Software License v2
+
+  (ASLv2) Apache Commons IO
+    The following NOTICE information applies:
+      Apache Commons IO
+      Copyright 2002-2016 The Apache Software Foundation
+
+  (ASLv2) Apache Commons Net
+    The following NOTICE information applies:
+      Apache Commons Net
+      Copyright 2001-2013 The Apache Software Foundation
+
+  (ASLv2) Apache Commons Collections
+    The following NOTICE information applies:
+      Apache Commons Collections
+      Copyright 2001-2013 The Apache Software Foundation	
+
+  (ASLv2) Apache Commons Compress
+    The following NOTICE information applies:
+      Apache Commons Compress
+      Copyright 2002-2017 The Apache Software Foundation
+
+      The files in the package org.apache.commons.compress.archivers.sevenz
+      were derived from the LZMA SDK, version 9.20 (C/ and CPP/7zip/),
+      which has been placed in the public domain:
+
+      "LZMA SDK is placed in the public domain." (http://www.7-zip.org/sdk.html)
+
+  (ASLv2) Jettison
+    The following NOTICE information applies:
+         Copyright 2006 Envoi Solutions LLC
+
+  (ASLv2) Apache Commons Codec
+    The following NOTICE information applies:
+      Apache Commons Codec
+      Copyright 2002-2014 The Apache Software Foundation
+
+      src/test/org/apache/commons/codec/language/DoubleMetaphoneTest.java
+      contains test data from http://aspell.net/test/orig/batch0.tab.
+      Copyright (C) 2002 Kevin Atkinson (kevina@gnu.org)
+
+      ===============================================================================
+
+      The content of package org.apache.commons.codec.language.bm has been translated
+      from the original php source code available at http://stevemorse.org/phoneticinfo.htm
+      with permission from the original authors.
+      Original source copyright:
+      Copyright (c) 2008 Alexander Beider & Stephen P. Morse.
+
+  (ASLv2) Apache HttpComponents
+    The following NOTICE information applies:
+      Apache HttpClient
+      Copyright 1999-2014 The Apache Software Foundation
+      
+      Apache HttpCore
+      Copyright 2005-2014 The Apache Software Foundation
+
+      This project contains annotations derived from JCIP-ANNOTATIONS
+      Copyright (c) 2005 Brian Goetz and Tim Peierls. See http://www.jcip.net
+
+  (ASLv2) Apache Commons Logging
+    The following NOTICE information applies:
+      Apache Commons Logging
+      Copyright 2003-2013 The Apache Software Foundation
+
+  (ASLv2) Apache Commons Lang
+    The following NOTICE information applies:
+      Apache Commons Lang
+      Copyright 2001-2015 The Apache Software Foundation
+
+      This product includes software from the Spring Framework,
+      under the Apache License 2.0 (see: StringUtils.containsWhitespace())
+
+  (ASLv2) Apache Commons Math
+    The following NOTICE information applies:
+      Apache Commons Math
+      Copyright 2001-2016 The Apache Software Foundation
+
+      This product includes software developed at
+      The Apache Software Foundation (http://www.apache.org/).
+
+      This product includes software developed for Orekit by
+      CS Systèmes d'Information (http://www.c-s.fr/)
+      Copyright 2010-2012 CS Systèmes d'Information
+
+  (ASLv2) Spring Framework
+    The following NOTICE information applies:
+      Spring Framework 4.1.6.RELEASE
+      Copyright (c) 2002-2015 Pivotal, Inc.
+
+  (ASLv2) Spring Security
+    The following NOTICE information applies:
+          Spring Framework 4.0.3.RELEASE
+          Copyright (c) 2002-2015 Pivotal, Inc.
+
+  (ASLv2) Swagger Core
+    The following NOTICE information applies:
+      Swagger Core 1.5.3-M1
+      Copyright 2015 Reverb Technologies, Inc.
+
+  (ASLv2) JSON-SMART
+    The following NOTICE information applies:
+      Copyright 2011 JSON-SMART authors
+
+  (ASLv2) JsonPath
+    The following NOTICE information applies:
+      Copyright 2011 JsonPath authors
+
+  (ASLv2) Quartz
+    The following NOTICE information applies:
+      Copyright Declaration:
+      Copyright © 2003-2016 Software AG, Darmstadt, Germany and/or Software AG USA Inc., Reston, VA, USA, and/or its subsidiaries and/or its affiliates and/or their licensors.
+
+      Trademark and Patent declaration
+      The name Software AG and all Software AG product names are either trademarks or registered trademarks of Software AG and/or Software AG USA Inc. and/or its subsidiaries and/or its affiliates
+      and/or their licensors. Other company and product names mentioned herein may be trademarks of their respective owners.
+
+      Detailed information on trademarks and patents owned by Software AG and/or its subsidiaries is located at http://softwareag.com/licenses.
+
+      Third Party declaration
+      This software may include portions of third-party products. For third-party copyright notices, license terms, additional rights or restrictions, please refer to "License Texts, Copyright
+      Notices and Disclaimers of Third Party Products". For certain specific third-party license restrictions, please refer to section E of the Legal Notices available under "License Terms and
+      Conditions for Use of Software AG Products / Copyright and Trademark Notices of Software AG Products". These documents are part of the product documentation, located at
+      http://softwareag.com/licenses and/or in the root installation directory of the licensed product(s).
+
+      Confidentiality Disclaimer:
+      Use, reproduction, transfer, publication or disclosure is prohibited except as specifically provided for in your License Agreement with Software AG.
+      Contact GitHub API Training Shop Blog About
+
+  (ASLv2) Jackson JSON processor
+    The following NOTICE information applies:
+      # Jackson JSON processor
+
+      Jackson is a high-performance, Free/Open Source JSON processing library.
+      It was originally written by Tatu Saloranta (tatu.saloranta@iki.fi), and has
+      been in development since 2007.
+      It is currently developed by a community of developers, as well as supported
+      commercially by FasterXML.com.
+
+      ## Licensing
+
+      Jackson core and extension components may licensed under different licenses.
+      To find the details that apply to this artifact see the accompanying LICENSE file.
+      For more information, including possible other licensing options, contact
+      FasterXML.com (http://fasterxml.com).
+
+      ## Credits
+
+      A list of contributors may be found from CREDITS file, which is included
+      in some artifacts (usually source distributions); but is always available
+      from the source code management (SCM) system project uses.
+
+  (ASLv2) Classmate
+    The following NOTICE information applies
+      Java ClassMate library was originally written by Tatu Saloranta (tatu.saloranta@iki.fi)
+
+      Other developers who have contributed code are:
+
+      * Brian Langel
+
+  (ASLv2) Caffeine
+    The following NOTICE information applies:
+      Caffeine (caching library)
+      Copyright Ben Manes
+
+************************
+Common Development and Distribution License 1.1
+************************
+
+The following binary components are provided under the Common Development and Distribution License 1.1. See project link for details.
+
+    (CDDL 1.1) (GPL2 w/ CPE) jersey-client (org.glassfish.jersey.core:jersey-client:jar:2.26 - https://jersey.github.io/)
+    (CDDL 1.1) (GPL2 w/ CPE) jersey-common (org.glassfish.jersey.core:jersey-common:jar:2.26 - https://jersey.github.io/)
+    (CDDL 1.1) (GPL2 w/ CPE) jersey-container-servlet-core (org.glassfish.jersey.containers:jersey-container-servlet-core:jar:2.26 - https://jersey.github.io/)
+    (CDDL 1.1) (GPL2 w/ CPE) jersey-entity-filtering (org.glassfish.jersey.ext:jersey-entity-filtering:jar:2.26 - https://jersey.github.io/)
+    (CDDL 1.1) (GPL2 w/ CPE) jersey-hk2 (org.glassfish.jersey.inject:jersey-hk2:jar:2.26 - https://jersey.github.io/)
+    (CDDL 1.1) (GPL2 w/ CPE) jersey-media-jaxb (org.glassfish.jersey.media:jersey-media-jaxb:jar:2.26 - https://jersey.github.io/)
+    (CDDL 1.1) (GPL2 w/ CPE) jersey-media-json-jackson (org.glassfish.jersey.media:jersey-media-json-jackson:jar:2.26 - https://jersey.github.io/)
+    (CDDL 1.1) (GPL2 w/ CPE) jersey-media-multipart (org.glassfish.jersey.media:jersey-media-multipart:jar:2.26 - https://jersey.github.io/)
+    (CDDL 1.1) (GPL2 w/ CPE) jersey-server (org.glassfish.jersey.core:jersey-server:jar:2.26 - https://jersey.github.io/)
+    (CDDL 1.1) (GPL2 w/ CPE) hk2 (org.glassfish.hk2:hk2:jar:2.5.0-b42 - https://javaee.github.io/glassfish/)
+    (CDDL 1.1) (GPL2 w/ CPE) hk2-api (org.glassfish.hk2:hk2-api:jar:2.5.0-b42 - https://javaee.github.io/glassfish/)
+    (CDDL 1.1) (GPL2 w/ CPE) hk2-utils (org.glassfish.hk2:hk2-utils:jar:2.5.0-b42 - https://javaee.github.io/glassfish/)
+    (CDDL 1.1) (GPL2 w/ CPE) hk2-locator (org.glassfish.hk2:hk2-locator:jar:2.5.0-b42 - https://javaee.github.io/glassfish/)
+    (CDDL 1.1) (GPL2 w/ CPE) hk2-config (org.glassfish.hk2:hk2-config:jar:2.5.0-b42 - https://javaee.github.io/glassfish/)
+    (CDDL 1.1) (GPL2 w/ CPE) hk2-core (org.glassfish.hk2:hk2-core:jar:2.5.0-b42 - https://javaee.github.io/glassfish/)
+    (CDDL 1.1) (GPL2 w/ CPE) hk2-runlevel (org.glassfish.hk2:hk2-runlevel:jar:2.5.0-b42 - https://javaee.github.io/glassfish/)
+    (CDDL 1.1) (GPL2 w/ CPE) config-types (org.glassfish.hk2:config-types:jar:2.5.0-b42 - https://javaee.github.io/glassfish/)
+    (CDDL 1.1) (GPL2 w/ CPE) class-model (org.glassfish.hk2:class-model:jar:2.5.0-b42 - https://javaee.github.io/glassfish/)
+    (CDDL 1.1) (GPL2 w/ CPE) asm-all-repackaged (org.glassfish.hk2.external:asm-all-repackaged:jar:2.5.0-b42 - https://javaee.github.io/glassfish/)
+    (CDDL 1.1) (GPL2 w/ CPE) aopalliance-repackaged (org.glassfish.hk2.external:aopalliance-repackaged:jar:2.5.0-b42 - https://javaee.github.io/glassfish/)
+    (CDDL 1.1) (GPL2 w/ CPE) javax.inject:1 as OSGi bundle (org.glassfish.hk2.external:javax.inject:jar:2.4.0-b25 - https://hk2.java.net/external/javax.inject)
+    (CDDL 1.1) (GPL2 w/ CPE) MIME Streaming Extension (org.jvnet.mimepull:mimepull:jar:1.9.3 - http://mimepull.java.net)
+    (CDDL 1.1) (GPL2 w/ CPE) JavaMail API (compat) (javax.mail:mail:jar:1.4.7 - http://kenai.com/projects/javamail/mail)
+    (CDDL 1.1) (GPL2 w/ CPE) javax.ws.rs-api (javax.ws.rs:javax.ws.rs-api:jar:2.1 - http://jax-rs-spec.java.net)
+
+************************
+Common Development and Distribution License 1.0
+************************
+
+The following binary components are provided under the Common Development and Distribution License 1.0.  See project link for details.
+
+    (CDDL 1.0) (GPL3) Streaming API For XML (javax.xml.stream:stax-api:jar:1.0-2 - no url provided)
+    (CDDL 1.0) JavaBeans Activation Framework (JAF) (javax.activation:activation:jar:1.1 - http://java.sun.com/products/javabeans/jaf/index.jsp)
+
+************************
+Eclipse Public License 1.0
+************************
+
+The following binary components are provided under the Eclipse Public License 1.0.  See project link for details.
+
+    (EPL 1.0) AspectJ Weaver (org.aspectj:aspectjweaver:jar:1.8.5 - http://www.aspectj.org)
+    (EPL 1.0)(MPL 2.0) H2 Database (com.h2database:h2:jar:1.3.176 - http://www.h2database.com/html/license.html)
+    (EPL 1.0)(LGPL 2.1) Logback Classic (ch.qos.logback:logback-classic:jar:1.2.3 - http://logback.qos.ch/)
+    (EPL 1.0)(LGPL 2.1) Logback Core (ch.qos.logback:logback-core:jar:1.2.3 - http://logback.qos.ch/)
+
+******************
+SIL OFL 1.1
+******************
+
+The following binary components are provided under the SIL Open Font License 1.1
+  (SIL OFL 1.1) FontAwesome (4.7.0 - https://fontawesome.com/license/free)
+
+
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-server-nar/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-server-nar/pom.xml
new file mode 100644
index 0000000..e65acde
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-server-nar/pom.xml
@@ -0,0 +1,198 @@
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <!--
+      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.
+    -->
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>nifi-framework-bundle</artifactId>
+        <version>1.13.0-SNAPSHOT</version>
+    </parent>
+    <artifactId>nifi-server-nar</artifactId>
+    <packaging>nar</packaging>
+    <description>NiFi: Web/UI Nar</description>
+    <properties>
+        <maven.javadoc.skip>true</maven.javadoc.skip>
+        <source.skip>true</source.skip>
+    </properties>
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-framework-nar</artifactId>
+            <version>1.13.0-SNAPSHOT</version>
+            <type>nar</type>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-jetty</artifactId>
+        </dependency>
+        <!-- Dependencies provided by nifi-framework-nar -->
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-framework-core</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-framework-core-api</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-framework-nar-utils</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-framework-authorization</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-framework-cluster</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-framework-cluster-protocol</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-web-security</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-web-utils</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-administration</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-security-utils-api</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-security-utils</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-user-actions</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>nifi-web-optimistic-locking</artifactId>
+        <scope>provided</scope>
+    </dependency>
+        <!-- Spring dependencies provided by nifi-framework-nar -->
+        <dependency>
+            <groupId>org.springframework</groupId>
+            <artifactId>spring-web</artifactId>
+            <version>${spring.version}</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.springframework</groupId>
+            <artifactId>spring-aop</artifactId>
+            <version>${spring.version}</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.springframework</groupId>
+            <artifactId>spring-beans</artifactId>
+            <version>${spring.version}</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.springframework</groupId>
+            <artifactId>spring-core</artifactId>
+            <version>${spring.version}</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.springframework</groupId>
+            <artifactId>spring-context</artifactId>
+            <version>${spring.version}</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.springframework</groupId>
+            <artifactId>spring-expression</artifactId>
+            <version>${spring.version}</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.springframework</groupId>
+            <artifactId>spring-jdbc</artifactId>
+            <version>${spring.version}</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.springframework</groupId>
+            <artifactId>spring-tx</artifactId>
+            <version>${spring.version}</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.springframework.security</groupId>
+            <artifactId>spring-security-core</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.springframework.security</groupId>
+            <artifactId>spring-security-config</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.springframework.security</groupId>
+            <artifactId>spring-security-web</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.springframework.security.kerberos</groupId>
+            <artifactId>spring-security-kerberos-core</artifactId>
+            <scope>provided</scope>
+        </dependency>
+
+        <!-- mark these nifi artifacts as provided since it is included in the lib -->
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-runtime</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-nar-utils</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-properties</artifactId>
+            <scope>provided</scope>
+        </dependency>
+    </dependencies>
+</project>
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-server-nar/src/main/resources/META-INF/LICENSE b/nifi-nar-bundles/nifi-framework-bundle/nifi-server-nar/src/main/resources/META-INF/LICENSE
new file mode 100644
index 0000000..c6530dd
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-server-nar/src/main/resources/META-INF/LICENSE
@@ -0,0 +1,880 @@
+
+                                 Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+   1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+   2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+   3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+   4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+   6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+   7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+   8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+   9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+
+   END OF TERMS AND CONDITIONS
+
+   APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "[]"
+      replaced with your own identifying information. (Don't include
+      the brackets!)  The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+   Copyright [yyyy] [name of copyright owner]
+
+   Licensed 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.
+
+APACHE NIFI SUBCOMPONENTS:
+
+The Apache NiFi project contains subcomponents with separate copyright
+notices and license terms. Your use of the source code for the these
+subcomponents is subject to the terms and conditions of the following
+licenses. 
+
+  The binary distribution of this product bundles 'Antlr 3' which is available
+  under a "3-clause BSD" license.  For details see http://www.antlr3.org/license.html
+
+    Copyright (c) 2010 Terence Parr
+    All rights reserved.
+
+    Redistribution and use in source and binary forms, with or without 
+    modification, are permitted provided that the following conditions are met:
+
+    Redistributions of source code must retain the above copyright notice, this 
+    list of conditions and the following disclaimer.
+    Redistributions in binary form must reproduce the above copyright notice, 
+    this list of conditions and the following disclaimer in the documentation 
+    and/or other materials provided with the distribution.
+    Neither the name of the author nor the names of its contributors may be used 
+    to endorse or promote products derived from this software without specific 
+    prior written permission.
+    THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" 
+    AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE 
+    IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE 
+    ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE 
+    LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR 
+    CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF 
+    SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS 
+    INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN 
+    CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) 
+    ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF 
+    THE POSSIBILITY OF SUCH DAMAGE.
+
+  The binary distribution of this product bundles 'Bouncy Castle JDK 1.5'
+  under an MIT style license.
+
+    Copyright (c) 2000 - 2015 The Legion of the Bouncy Castle Inc. (http://www.bouncycastle.org)
+
+    Permission is hereby granted, free of charge, to any person obtaining a copy
+    of this software and associated documentation files (the "Software"), to deal
+    in the Software without restriction, including without limitation the rights
+    to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+    copies of the Software, and to permit persons to whom the Software is
+    furnished to do so, subject to the following conditions:
+
+    The above copyright notice and this permission notice shall be included in
+    all copies or substantial portions of the Software.
+
+    THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+    IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+    FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+    AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+    LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+    OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
+    THE SOFTWARE.
+
+This product bundles 'Javascript D3 Library' which is available under a
+"3-clause BSD" license.
+
+    Copyright 2010-2017 Mike Bostock
+    All rights reserved.
+
+    Redistribution and use in source and binary forms, with or without modification,
+    are permitted provided that the following conditions are met:
+
+    * Redistributions of source code must retain the above copyright notice, this
+      list of conditions and the following disclaimer.
+
+    * Redistributions in binary form must reproduce the above copyright notice,
+      this list of conditions and the following disclaimer in the documentation
+      and/or other materials provided with the distribution.
+
+    * Neither the name of the author nor the names of contributors may be used to
+      endorse or promote products derived from this software without specific prior
+      written permission.
+
+    THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
+    ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
+    WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+    DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR
+    ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
+    (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+    LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON
+    ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+    (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+    SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+This product bundles 'Javascript D3 Selection Multi Library' which is available under a
+"3-clause BSD" license.
+
+    Copyright (c) 2010-2016, Michael Bostock
+    All rights reserved.
+
+    Redistribution and use in source and binary forms, with or without
+    modification, are permitted provided that the following conditions are met:
+
+    * Redistributions of source code must retain the above copyright notice, this
+      list of conditions and the following disclaimer.
+
+    * Redistributions in binary form must reproduce the above copyright notice,
+      this list of conditions and the following disclaimer in the documentation
+      and/or other materials provided with the distribution.
+
+    * The name Michael Bostock may not be used to endorse or promote products
+      derived from this software without specific prior written permission.
+
+    THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
+    AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
+    IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+    DISCLAIMED. IN NO EVENT SHALL MICHAEL BOSTOCK BE LIABLE FOR ANY DIRECT,
+    INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING,
+    BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+    DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY
+    OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING
+    NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE,
+    EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+This product bundles 'CodeMirror' which is available under an MIT style license.
+
+    Copyright (C) 2014 by Marijn Haverbeke <ma...@gmail.com> and others
+
+    Permission is hereby granted, free of charge, to any person obtaining a copy
+    of this software and associated documentation files (the "Software"), to deal
+    in the Software without restriction, including without limitation the rights
+    to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+    copies of the Software, and to permit persons to whom the Software is
+    furnished to do so, subject to the following conditions:
+
+    The above copyright notice and this permission notice shall be included in
+    all copies or substantial portions of the Software.
+
+    THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+    IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+    FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+    AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+    LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+    OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
+    THE SOFTWARE.
+
+This product bundles 'JQuery' which is available under and MIT style license.
+    (c) 2005, 2014 jQuery Foundation, Inc.
+
+    Permission is hereby granted, free of charge, to any person obtaining a copy
+    of this software and associated documentation files (the "Software"), to deal
+    in the Software without restriction, including without limitation the rights
+    to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+    copies of the Software, and to permit persons to whom the Software is
+    furnished to do so, subject to the following conditions:
+
+    The above copyright notice and this permission notice shall be included in
+    all copies or substantial portions of the Software.
+
+    THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+    IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+    FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+    AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+    LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+    OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
+    THE SOFTWARE.
+
+This product bundles 'JQuery Event Drag' which is available under an MIT style
+license.
+    Copyright (c) 2008-2015 ThreeDubMedia
+
+    Permission is hereby granted, free of charge, to any person obtaining a copy
+    of this software and associated documentation files (the "Software"), to deal
+    in the Software without restriction, including without limitation the rights
+    to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+    copies of the Software, and to permit persons to whom the Software is
+    furnished to do so, subject to the following conditions:
+
+    The above copyright notice and this permission notice shall be included in
+    all copies or substantial portions of the Software.
+
+    THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+    IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+    FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+    AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+    LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+    OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
+    THE SOFTWARE.
+
+This product bundles 'jQuery Form Plugin' which is available under either the MIT
+or GPL license.  The license in effect here is the MIT license
+
+    Copyright 2006-2013 (c) M. Alsup
+
+    Permission is hereby granted, free of charge, to any person obtaining
+    a copy of this software and associated documentation files (the
+    "Software"), to deal in the Software without restriction, including
+    without limitation the rights to use, copy, modify, merge, publish,
+    distribute, sublicense, and/or sell copies of the Software, and to
+    permit persons to whom the Software is furnished to do so, subject to
+    the following conditions:
+
+    The above copyright notice and this permission notice shall be
+    included in all copies or substantial portions of the Software.
+
+    THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND,
+    EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF
+    MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND
+    NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE
+    LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION
+    OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION
+    WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
+
+This product bundles 'jQuery UI' which is available under an MIT style license.
+For details see http://jqueryui.com
+
+    Copyright 2014 jQuery Foundation and other contributors
+
+    Permission is hereby granted, free of charge, to any person obtaining
+    a copy of this software and associated documentation files (the
+    "Software"), to deal in the Software without restriction, including
+    without limitation the rights to use, copy, modify, merge, publish,
+    distribute, sublicense, and/or sell copies of the Software, and to
+    permit persons to whom the Software is furnished to do so, subject to
+    the following conditions:
+
+    The above copyright notice and this permission notice shall be
+    included in all copies or substantial portions of the Software.
+
+    THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND,
+    EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF
+    MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND
+    NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE
+    LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION
+    OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION
+    WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
+
+This product bundles 'jquery.base64.js' which is available under an MIT style license.
+
+    Copyright (c) 2013 Yannick Albert (http://yckart.com/)
+
+    Permission is hereby granted, free of charge, to any person obtaining
+    a copy of this software and associated documentation files (the "Software"),
+    to deal in the Software without restriction, including without limitation
+    the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or
+    sell copies of the Software, and to permit persons to whom the Software is
+    furnished to do so, subject to the following conditions:
+
+    The above copyright notice and this permission notice shall be
+    included in all copies or substantial portions of the Software.
+
+    THE SOFTWARE IS PROVIDED 'AS IS', WITHOUT WARRANTY OF ANY KIND,
+    EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF
+    MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND
+    NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE
+    LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION
+    OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION
+    WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
+
+This product bundles 'SlickGrid v2.3' which is available under an MIT style license.
+
+    Copyright (c) 2009-2016 Michael Leibman, http://github.com/mleibman/slickgrid
+
+    Permission is hereby granted, free of charge, to any person obtaining
+    a copy of this software and associated documentation files (the
+    "Software"), to deal in the Software without restriction, including
+    without limitation the rights to use, copy, modify, merge, publish,
+    distribute, sublicense, and/or sell copies of the Software, and to
+    permit persons to whom the Software is furnished to do so, subject to
+    the following conditions:
+
+    The above copyright notice and this permission notice shall be
+    included in all copies or substantial portions of the Software.
+
+    THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND,
+    EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF
+    MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND
+    NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE
+    LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION
+    OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION
+    WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
+
+This product bundles 'qTip2' which is available under an MIT style license.
+For details see http://qtip2.com
+
+    Copyright (c) 2012 Craig Michael Thompson
+
+    Permission is hereby granted, free of charge, to any person
+    obtaining a copy of this software and associated documentation
+    files (the "Software"), to deal in the Software without
+    restriction, including without limitation the rights to use,
+    copy, modify, merge, publish, distribute, sublicense, and/or sell
+    copies of the Software, and to permit persons to whom the
+    Software is furnished to do so, subject to the following
+    conditions:
+
+    The above copyright notice and this permission notice shall be
+    included in all copies or substantial portions of the Software.
+
+    THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND,
+    EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES
+    OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND
+    NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT
+    HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY,
+    WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING
+    FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR
+    OTHER DEALINGS IN THE SOFTWARE.
+
+This product bundles 'url-search-params' which is available under an MIT style license.
+For details see https://github.com/WebReflection/url-search-params
+
+    Copyright (C) 2015 by WebReflection
+
+    Permission is hereby granted, free of charge, to any person obtaining a copy
+    of this software and associated documentation files (the "Software"), to deal
+    in the Software without restriction, including without limitation the rights
+    to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+    copies of the Software, and to permit persons to whom the Software is
+    furnished to do so, subject to the following conditions:
+
+    The above copyright notice and this permission notice shall be included in
+    all copies or substantial portions of the Software.
+
+    THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+    IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+    FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+    AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+    LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+    OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
+    THE SOFTWARE.
+
+This product bundles 'jQuery MiniColors' which is available under the MIT License.
+For details see http://www.abeautifulsite.net/
+
+    Copyright Cory LaViska for A Beautiful Site, LLC. (http://www.abeautifulsite.net/)
+
+    Permission is hereby granted, free of charge, to any person
+    obtaining a copy of this software and associated documentation
+    files (the "Software"), to deal in the Software without
+    restriction, including without limitation the rights to use,
+    copy, modify, merge, publish, distribute, sublicense, and/or sell
+    copies of the Software, and to permit persons to whom the
+    Software is furnished to do so, subject to the following
+    conditions:
+
+    The above copyright notice and this permission notice shall be
+    included in all copies or substantial portions of the Software.
+
+    THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND,
+    EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES
+    OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND
+    NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT
+    HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY,
+    WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING
+    FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR
+    OTHER DEALINGS IN THE SOFTWARE.
+
+This product bundles 'json2.js' which is available in the 'public domain'.
+    For details see https://github.com/douglascrockford/JSON-js
+
+This product bundles 'reset.css' which is available in the 'public domain'.
+    For details see http://meyerweb.com/eric/tools/css/reset/
+
+This product bundles 'Angular' which is available under an MIT license.
+
+    Copyright (c) 2010-2016 Google, Inc. http://angularjs.org
+
+    Permission is hereby granted, free of charge, to any person obtaining a copy
+    of this software and associated documentation files (the "Software"), to deal
+    in the Software without restriction, including without limitation the rights
+    to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+    copies of the Software, and to permit persons to whom the Software is
+    furnished to do so, subject to the following conditions:
+
+    The above copyright notice and this permission notice shall be included in
+    all copies or substantial portions of the Software.
+
+    THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+    IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+    FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+    AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+    LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+    OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
+    THE SOFTWARE.
+
+This product bundles 'Angular Material' which is available under an MIT license.
+
+    Copyright (c) 2014-2016 Google, Inc. http://angularjs.org
+
+    Permission is hereby granted, free of charge, to any person obtaining a copy
+    of this software and associated documentation files (the "Software"), to deal
+    in the Software without restriction, including without limitation the rights
+    to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+    copies of the Software, and to permit persons to whom the Software is
+    furnished to do so, subject to the following conditions:
+
+    The above copyright notice and this permission notice shall be included in
+    all copies or substantial portions of the Software.
+
+    THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+    IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+    FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+    AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+    LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+    OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
+    THE SOFTWARE.
+
+This product bundles 'Angular Aria' which is available under an MIT license.
+
+    Copyright (c) 2010-2016 Google, Inc. http://angularjs.org
+
+    Permission is hereby granted, free of charge, to any person obtaining a copy
+    of this software and associated documentation files (the "Software"), to deal
+    in the Software without restriction, including without limitation the rights
+    to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+    copies of the Software, and to permit persons to whom the Software is
+    furnished to do so, subject to the following conditions:
+
+    The above copyright notice and this permission notice shall be included in
+    all copies or substantial portions of the Software.
+
+    THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+    IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+    FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+    AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+    LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+    OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
+
+This product bundles 'Angular Animate' which is available under an MIT license.
+
+    Copyright (c) 2010-2016 Google, Inc. http://angularjs.org
+
+    Permission is hereby granted, free of charge, to any person obtaining a copy
+    of this software and associated documentation files (the "Software"), to deal
+    in the Software without restriction, including without limitation the rights
+    to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+    copies of the Software, and to permit persons to whom the Software is
+    furnished to do so, subject to the following conditions:
+
+    The above copyright notice and this permission notice shall be included in
+    all copies or substantial portions of the Software.
+
+    THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+    IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+    FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+    AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+    LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+    OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
+
+This product bundles 'Angular Route' which is available under an MIT license.
+
+    Copyright (c) 2010-2016 Google, Inc. http://angularjs.org
+
+    Permission is hereby granted, free of charge, to any person obtaining a copy
+    of this software and associated documentation files (the "Software"), to deal
+    in the Software without restriction, including without limitation the rights
+    to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+    copies of the Software, and to permit persons to whom the Software is
+    furnished to do so, subject to the following conditions:
+
+    The above copyright notice and this permission notice shall be included in
+    all copies or substantial portions of the Software.
+
+    THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+    IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+    FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+    AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+    LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+    OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
+
+This product bundles 'Angular Resource' which is available under an MIT license.
+
+    Copyright (c) 2010-2016 Google, Inc. http://angularjs.org
+
+    Permission is hereby granted, free of charge, to any person obtaining a copy
+    of this software and associated documentation files (the "Software"), to deal
+    in the Software without restriction, including without limitation the rights
+    to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+    copies of the Software, and to permit persons to whom the Software is
+    furnished to do so, subject to the following conditions:
+
+    The above copyright notice and this permission notice shall be included in
+    all copies or substantial portions of the Software.
+
+    THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+    IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+    FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+    AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+    LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+    OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
+
+This product bundles 'Angular Sanitize' which is available under an MIT license.
+
+    Copyright (c) 2010-2016 Google, Inc. http://angularjs.org
+
+    Permission is hereby granted, free of charge, to any person obtaining a copy
+    of this software and associated documentation files (the "Software"), to deal
+    in the Software without restriction, including without limitation the rights
+    to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+    copies of the Software, and to permit persons to whom the Software is
+    furnished to do so, subject to the following conditions:
+
+    The above copyright notice and this permission notice shall be included in
+    all copies or substantial portions of the Software.
+
+    THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+    IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+    FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+    AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+    LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+    OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
+
+This product bundles 'Angular Messages' which is available under an MIT license.
+
+    Copyright (c) 2010-2016 Google, Inc. http://angularjs.org
+
+    Permission is hereby granted, free of charge, to any person obtaining a copy
+    of this software and associated documentation files (the "Software"), to deal
+    in the Software without restriction, including without limitation the rights
+    to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+    copies of the Software, and to permit persons to whom the Software is
+    furnished to do so, subject to the following conditions:
+
+    The above copyright notice and this permission notice shall be included in
+    all copies or substantial portions of the Software.
+
+    THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+    IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+    FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+    AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+    LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+    OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
+
+This product bundles 'Fontello' which is available under an MIT license.
+
+    Copyright (C) 2011 by Vitaly Puzrin
+
+    Permission is hereby granted, free of charge, to any person obtaining a copy
+    of this software and associated documentation files (the "Software"), to deal
+    in the Software without restriction, including without limitation the rights
+    to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+    copies of the Software, and to permit persons to whom the Software is
+    furnished to do so, subject to the following conditions:
+
+    The above copyright notice and this permission notice shall be included in
+    all copies or substantial portions of the Software.
+
+    THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+    IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+    FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+    AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+    LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+    OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
+    THE SOFTWARE.
+
+This product bundles 'jsonlint' which is available under an MIT license.
+
+    Copyright (C) 2012 Zachary Carter
+
+    Permission is hereby granted, free of charge, to any person obtaining a copy
+    of this software and associated documentation files (the "Software"), to deal
+    in the Software without restriction, including without limitation the rights
+    to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+    copies of the Software, and to permit persons to whom the Software is
+    furnished to do so, subject to the following conditions:
+
+    The above copyright notice and this permission notice shall be included in
+    all copies or substantial portions of the Software.
+
+    THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+    IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+    FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+    AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+    LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+    OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
+    THE SOFTWARE.
+
+This product bundles 'asm' which is available under a 3-Clause BSD style license.
+For details see http://asm.ow2.org/asmdex-license.html
+
+    Copyright (c) 2012 France Télécom
+    All rights reserved.
+
+    Redistribution and use in source and binary forms, with or without
+    modification, are permitted provided that the following conditions
+    are met:
+    1. Redistributions of source code must retain the above copyright
+       notice, this list of conditions and the following disclaimer.
+    2. Redistributions in binary form must reproduce the above copyright
+       notice, this list of conditions and the following disclaimer in the
+       documentation and/or other materials provided with the distribution.
+    3. Neither the name of the copyright holders nor the names of its
+       contributors may be used to endorse or promote products derived from
+       this software without specific prior written permission.
+
+    THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
+    AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
+    IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
+    ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE
+    LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
+    CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
+    SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
+    INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
+    CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
+    ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF
+    THE POSSIBILITY OF SUCH DAMAGE.
+
+This product bundles 'lodash' which is available under an MIT license.
+
+    Copyright JS Foundation and other contributors <https://js.foundation/>
+
+    Based on Underscore.js, copyright Jeremy Ashkenas,
+    DocumentCloud and Investigative Reporters & Editors <http://underscorejs.org/>
+
+    This software consists of voluntary contributions made by many
+    individuals. For exact contribution history, see the revision history
+    available at https://github.com/lodash/lodash
+
+    The following license applies to all parts of this software except as
+    documented below:
+
+    ====
+
+    Permission is hereby granted, free of charge, to any person obtaining
+    a copy of this software and associated documentation files (the
+    "Software"), to deal in the Software without restriction, including
+    without limitation the rights to use, copy, modify, merge, publish,
+    distribute, sublicense, and/or sell copies of the Software, and to
+    permit persons to whom the Software is furnished to do so, subject to
+    the following conditions:
+
+    The above copyright notice and this permission notice shall be
+    included in all copies or substantial portions of the Software.
+
+    THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND,
+    EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF
+    MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND
+    NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE
+    LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION
+    OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION
+    WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
+
+    ====
+
+    Copyright and related rights for sample code are waived via CC0. Sample
+    code is defined as all source code displayed within the prose of the
+    documentation.
+
+    CC0: http://creativecommons.org/publicdomain/zero/1.0/
+
+    ====
+
+    Files located in the node_modules and vendor directories are externally
+    maintained libraries used by this software which have their own
+    licenses; we recommend you read them, as their terms may differ from the
+    terms above.
+
+This product bundles 'moment' which is available under an MIT license.
+
+    Copyright (c) JS Foundation and other contributors
+
+    Permission is hereby granted, free of charge, to any person
+    obtaining a copy of this software and associated documentation
+    files (the "Software"), to deal in the Software without
+    restriction, including without limitation the rights to use,
+    copy, modify, merge, publish, distribute, sublicense, and/or sell
+    copies of the Software, and to permit persons to whom the
+    Software is furnished to do so, subject to the following
+    conditions:
+
+    The above copyright notice and this permission notice shall be
+    included in all copies or substantial portions of the Software.
+
+    THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND,
+    EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES
+    OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND
+    NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT
+    HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY,
+    WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING
+    FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR
+    OTHER DEALINGS IN THE SOFTWARE.
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-server-nar/src/main/resources/META-INF/NOTICE b/nifi-nar-bundles/nifi-framework-bundle/nifi-server-nar/src/main/resources/META-INF/NOTICE
new file mode 100644
index 0000000..5a88b5d
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-server-nar/src/main/resources/META-INF/NOTICE
@@ -0,0 +1,234 @@
+nifi-server-nar
+Copyright 2014-2020 The Apache Software Foundation
+
+This product includes software developed at
+The Apache Software Foundation (http://www.apache.org/).
+
+******************
+Apache Software License v2
+******************
+
+The following binary components are provided under the Apache Software License v2
+
+  (ASLv2) Apache Commons IO
+    The following NOTICE information applies:
+      Apache Commons IO
+      Copyright 2002-2016 The Apache Software Foundation
+
+  (ASLv2) Apache Commons Net
+    The following NOTICE information applies:
+      Apache Commons Net
+      Copyright 2001-2013 The Apache Software Foundation
+
+  (ASLv2) Apache Commons Collections
+    The following NOTICE information applies:
+      Apache Commons Collections
+      Copyright 2001-2013 The Apache Software Foundation	
+
+  (ASLv2) Apache Commons Compress
+    The following NOTICE information applies:
+      Apache Commons Compress
+      Copyright 2002-2017 The Apache Software Foundation
+
+      The files in the package org.apache.commons.compress.archivers.sevenz
+      were derived from the LZMA SDK, version 9.20 (C/ and CPP/7zip/),
+      which has been placed in the public domain:
+
+      "LZMA SDK is placed in the public domain." (http://www.7-zip.org/sdk.html)
+
+  (ASLv2) Jettison
+    The following NOTICE information applies:
+         Copyright 2006 Envoi Solutions LLC
+
+  (ASLv2) Apache Commons Codec
+    The following NOTICE information applies:
+      Apache Commons Codec
+      Copyright 2002-2014 The Apache Software Foundation
+
+      src/test/org/apache/commons/codec/language/DoubleMetaphoneTest.java
+      contains test data from http://aspell.net/test/orig/batch0.tab.
+      Copyright (C) 2002 Kevin Atkinson (kevina@gnu.org)
+
+      ===============================================================================
+
+      The content of package org.apache.commons.codec.language.bm has been translated
+      from the original php source code available at http://stevemorse.org/phoneticinfo.htm
+      with permission from the original authors.
+      Original source copyright:
+      Copyright (c) 2008 Alexander Beider & Stephen P. Morse.
+
+  (ASLv2) Apache HttpComponents
+    The following NOTICE information applies:
+      Apache HttpClient
+      Copyright 1999-2014 The Apache Software Foundation
+      
+      Apache HttpCore
+      Copyright 2005-2014 The Apache Software Foundation
+
+      This project contains annotations derived from JCIP-ANNOTATIONS
+      Copyright (c) 2005 Brian Goetz and Tim Peierls. See http://www.jcip.net
+
+  (ASLv2) Apache Commons Logging
+    The following NOTICE information applies:
+      Apache Commons Logging
+      Copyright 2003-2013 The Apache Software Foundation
+
+  (ASLv2) Apache Commons Lang
+    The following NOTICE information applies:
+      Apache Commons Lang
+      Copyright 2001-2015 The Apache Software Foundation
+
+      This product includes software from the Spring Framework,
+      under the Apache License 2.0 (see: StringUtils.containsWhitespace())
+
+  (ASLv2) Apache Commons Math
+    The following NOTICE information applies:
+      Apache Commons Math
+      Copyright 2001-2016 The Apache Software Foundation
+
+      This product includes software developed at
+      The Apache Software Foundation (http://www.apache.org/).
+
+      This product includes software developed for Orekit by
+      CS Systèmes d'Information (http://www.c-s.fr/)
+      Copyright 2010-2012 CS Systèmes d'Information
+
+  (ASLv2) Spring Framework
+    The following NOTICE information applies:
+      Spring Framework 4.1.6.RELEASE
+      Copyright (c) 2002-2015 Pivotal, Inc.
+
+  (ASLv2) Spring Security
+    The following NOTICE information applies:
+          Spring Framework 4.0.3.RELEASE
+          Copyright (c) 2002-2015 Pivotal, Inc.
+
+  (ASLv2) Swagger Core
+    The following NOTICE information applies:
+      Swagger Core 1.5.3-M1
+      Copyright 2015 Reverb Technologies, Inc.
+
+  (ASLv2) JSON-SMART
+    The following NOTICE information applies:
+      Copyright 2011 JSON-SMART authors
+
+  (ASLv2) JsonPath
+    The following NOTICE information applies:
+      Copyright 2011 JsonPath authors
+
+  (ASLv2) Quartz
+    The following NOTICE information applies:
+      Copyright Declaration:
+      Copyright © 2003-2016 Software AG, Darmstadt, Germany and/or Software AG USA Inc., Reston, VA, USA, and/or its subsidiaries and/or its affiliates and/or their licensors.
+
+      Trademark and Patent declaration
+      The name Software AG and all Software AG product names are either trademarks or registered trademarks of Software AG and/or Software AG USA Inc. and/or its subsidiaries and/or its affiliates
+      and/or their licensors. Other company and product names mentioned herein may be trademarks of their respective owners.
+
+      Detailed information on trademarks and patents owned by Software AG and/or its subsidiaries is located at http://softwareag.com/licenses.
+
+      Third Party declaration
+      This software may include portions of third-party products. For third-party copyright notices, license terms, additional rights or restrictions, please refer to "License Texts, Copyright
+      Notices and Disclaimers of Third Party Products". For certain specific third-party license restrictions, please refer to section E of the Legal Notices available under "License Terms and
+      Conditions for Use of Software AG Products / Copyright and Trademark Notices of Software AG Products". These documents are part of the product documentation, located at
+      http://softwareag.com/licenses and/or in the root installation directory of the licensed product(s).
+
+      Confidentiality Disclaimer:
+      Use, reproduction, transfer, publication or disclosure is prohibited except as specifically provided for in your License Agreement with Software AG.
+      Contact GitHub API Training Shop Blog About
+
+  (ASLv2) Jackson JSON processor
+    The following NOTICE information applies:
+      # Jackson JSON processor
+
+      Jackson is a high-performance, Free/Open Source JSON processing library.
+      It was originally written by Tatu Saloranta (tatu.saloranta@iki.fi), and has
+      been in development since 2007.
+      It is currently developed by a community of developers, as well as supported
+      commercially by FasterXML.com.
+
+      ## Licensing
+
+      Jackson core and extension components may licensed under different licenses.
+      To find the details that apply to this artifact see the accompanying LICENSE file.
+      For more information, including possible other licensing options, contact
+      FasterXML.com (http://fasterxml.com).
+
+      ## Credits
+
+      A list of contributors may be found from CREDITS file, which is included
+      in some artifacts (usually source distributions); but is always available
+      from the source code management (SCM) system project uses.
+
+  (ASLv2) Classmate
+    The following NOTICE information applies
+      Java ClassMate library was originally written by Tatu Saloranta (tatu.saloranta@iki.fi)
+
+      Other developers who have contributed code are:
+
+      * Brian Langel
+
+  (ASLv2) Caffeine
+    The following NOTICE information applies:
+      Caffeine (caching library)
+      Copyright Ben Manes
+
+************************
+Common Development and Distribution License 1.1
+************************
+
+The following binary components are provided under the Common Development and Distribution License 1.1. See project link for details.
+
+    (CDDL 1.1) (GPL2 w/ CPE) jersey-client (org.glassfish.jersey.core:jersey-client:jar:2.26 - https://jersey.github.io/)
+    (CDDL 1.1) (GPL2 w/ CPE) jersey-common (org.glassfish.jersey.core:jersey-common:jar:2.26 - https://jersey.github.io/)
+    (CDDL 1.1) (GPL2 w/ CPE) jersey-container-servlet-core (org.glassfish.jersey.containers:jersey-container-servlet-core:jar:2.26 - https://jersey.github.io/)
+    (CDDL 1.1) (GPL2 w/ CPE) jersey-entity-filtering (org.glassfish.jersey.ext:jersey-entity-filtering:jar:2.26 - https://jersey.github.io/)
+    (CDDL 1.1) (GPL2 w/ CPE) jersey-hk2 (org.glassfish.jersey.inject:jersey-hk2:jar:2.26 - https://jersey.github.io/)
+    (CDDL 1.1) (GPL2 w/ CPE) jersey-media-jaxb (org.glassfish.jersey.media:jersey-media-jaxb:jar:2.26 - https://jersey.github.io/)
+    (CDDL 1.1) (GPL2 w/ CPE) jersey-media-json-jackson (org.glassfish.jersey.media:jersey-media-json-jackson:jar:2.26 - https://jersey.github.io/)
+    (CDDL 1.1) (GPL2 w/ CPE) jersey-media-multipart (org.glassfish.jersey.media:jersey-media-multipart:jar:2.26 - https://jersey.github.io/)
+    (CDDL 1.1) (GPL2 w/ CPE) jersey-server (org.glassfish.jersey.core:jersey-server:jar:2.26 - https://jersey.github.io/)
+    (CDDL 1.1) (GPL2 w/ CPE) hk2 (org.glassfish.hk2:hk2:jar:2.5.0-b42 - https://javaee.github.io/glassfish/)
+    (CDDL 1.1) (GPL2 w/ CPE) hk2-api (org.glassfish.hk2:hk2-api:jar:2.5.0-b42 - https://javaee.github.io/glassfish/)
+    (CDDL 1.1) (GPL2 w/ CPE) hk2-utils (org.glassfish.hk2:hk2-utils:jar:2.5.0-b42 - https://javaee.github.io/glassfish/)
+    (CDDL 1.1) (GPL2 w/ CPE) hk2-locator (org.glassfish.hk2:hk2-locator:jar:2.5.0-b42 - https://javaee.github.io/glassfish/)
+    (CDDL 1.1) (GPL2 w/ CPE) hk2-config (org.glassfish.hk2:hk2-config:jar:2.5.0-b42 - https://javaee.github.io/glassfish/)
+    (CDDL 1.1) (GPL2 w/ CPE) hk2-core (org.glassfish.hk2:hk2-core:jar:2.5.0-b42 - https://javaee.github.io/glassfish/)
+    (CDDL 1.1) (GPL2 w/ CPE) hk2-runlevel (org.glassfish.hk2:hk2-runlevel:jar:2.5.0-b42 - https://javaee.github.io/glassfish/)
+    (CDDL 1.1) (GPL2 w/ CPE) config-types (org.glassfish.hk2:config-types:jar:2.5.0-b42 - https://javaee.github.io/glassfish/)
+    (CDDL 1.1) (GPL2 w/ CPE) class-model (org.glassfish.hk2:class-model:jar:2.5.0-b42 - https://javaee.github.io/glassfish/)
+    (CDDL 1.1) (GPL2 w/ CPE) asm-all-repackaged (org.glassfish.hk2.external:asm-all-repackaged:jar:2.5.0-b42 - https://javaee.github.io/glassfish/)
+    (CDDL 1.1) (GPL2 w/ CPE) aopalliance-repackaged (org.glassfish.hk2.external:aopalliance-repackaged:jar:2.5.0-b42 - https://javaee.github.io/glassfish/)
+    (CDDL 1.1) (GPL2 w/ CPE) javax.inject:1 as OSGi bundle (org.glassfish.hk2.external:javax.inject:jar:2.4.0-b25 - https://hk2.java.net/external/javax.inject)
+    (CDDL 1.1) (GPL2 w/ CPE) MIME Streaming Extension (org.jvnet.mimepull:mimepull:jar:1.9.3 - http://mimepull.java.net)
+    (CDDL 1.1) (GPL2 w/ CPE) JavaMail API (compat) (javax.mail:mail:jar:1.4.7 - http://kenai.com/projects/javamail/mail)
+    (CDDL 1.1) (GPL2 w/ CPE) javax.ws.rs-api (javax.ws.rs:javax.ws.rs-api:jar:2.1 - http://jax-rs-spec.java.net)
+
+************************
+Common Development and Distribution License 1.0
+************************
+
+The following binary components are provided under the Common Development and Distribution License 1.0.  See project link for details.
+
+    (CDDL 1.0) (GPL3) Streaming API For XML (javax.xml.stream:stax-api:jar:1.0-2 - no url provided)
+    (CDDL 1.0) JavaBeans Activation Framework (JAF) (javax.activation:activation:jar:1.1 - http://java.sun.com/products/javabeans/jaf/index.jsp)
+
+************************
+Eclipse Public License 1.0
+************************
+
+The following binary components are provided under the Eclipse Public License 1.0.  See project link for details.
+
+    (EPL 1.0) AspectJ Weaver (org.aspectj:aspectjweaver:jar:1.8.5 - http://www.aspectj.org)
+    (EPL 1.0)(MPL 2.0) H2 Database (com.h2database:h2:jar:1.3.176 - http://www.h2database.com/html/license.html)
+    (EPL 1.0)(LGPL 2.1) Logback Classic (ch.qos.logback:logback-classic:jar:1.2.3 - http://logback.qos.ch/)
+    (EPL 1.0)(LGPL 2.1) Logback Core (ch.qos.logback:logback-core:jar:1.2.3 - http://logback.qos.ch/)
+
+******************
+SIL OFL 1.1
+******************
+
+The following binary components are provided under the SIL Open Font License 1.1
+  (SIL OFL 1.1) FontAwesome (4.7.0 - https://fontawesome.com/license/free)
+
+
diff --git a/nifi-nar-bundles/nifi-framework-bundle/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/pom.xml
index d09beba..022b723 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/pom.xml
@@ -30,6 +30,8 @@
     <modules>
         <module>nifi-framework</module>
         <module>nifi-framework-nar</module>
+        <module>nifi-server-nar</module>
+        <module>nifi-headless-server-nar</module>
     </modules>
     <dependencyManagement>
         <dependencies>
diff --git a/nifi-nar-bundles/nifi-update-attribute-bundle/nifi-update-attribute-nar/pom.xml b/nifi-nar-bundles/nifi-update-attribute-bundle/nifi-update-attribute-nar/pom.xml
index 94111e4..e29890b 100644
--- a/nifi-nar-bundles/nifi-update-attribute-bundle/nifi-update-attribute-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-update-attribute-bundle/nifi-update-attribute-nar/pom.xml
@@ -28,6 +28,12 @@
     <dependencies>
         <dependency>
             <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-jetty-bundle</artifactId>
+            <version>1.13.0-SNAPSHOT</version>
+            <type>nar</type>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-processor-utils</artifactId>
             <version>1.13.0-SNAPSHOT</version>
         </dependency>
diff --git a/nifi-nar-bundles/pom.xml b/nifi-nar-bundles/pom.xml
index 0fdb2ab..12bbb28 100755
--- a/nifi-nar-bundles/pom.xml
+++ b/nifi-nar-bundles/pom.xml
@@ -303,6 +303,12 @@
             </dependency>
             <dependency>
                 <groupId>org.apache.nifi</groupId>
+                <artifactId>nifi-server-api</artifactId>
+                <version>1.13.0-SNAPSHOT</version>
+                <scope>provided</scope>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-runtime</artifactId>
                 <version>1.13.0-SNAPSHOT</version>
                 <scope>provided</scope>
diff --git a/nifi-server-api/pom.xml b/nifi-server-api/pom.xml
new file mode 100644
index 0000000..36b2d13
--- /dev/null
+++ b/nifi-server-api/pom.xml
@@ -0,0 +1,46 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+      http://www.apache.org/licenses/LICENSE-2.0
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>nifi</artifactId>
+        <version>1.13.0-SNAPSHOT</version>
+    </parent>
+    <artifactId>nifi-server-api</artifactId>
+    <packaging>jar</packaging>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+            <version>1.13.0-SNAPSHOT</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-framework-api</artifactId>
+            <version>1.13.0-SNAPSHOT</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-properties</artifactId>
+            <version>1.13.0-SNAPSHOT</version>
+            <scope>provided</scope>
+        </dependency>
+    </dependencies>
+</project>
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/main/java/org/apache/nifi/NiFiServer.java b/nifi-server-api/src/main/java/org/apache/nifi/NiFiServer.java
similarity index 87%
rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/main/java/org/apache/nifi/NiFiServer.java
rename to nifi-server-api/src/main/java/org/apache/nifi/NiFiServer.java
index 2169579..8124876 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/main/java/org/apache/nifi/NiFiServer.java
+++ b/nifi-server-api/src/main/java/org/apache/nifi/NiFiServer.java
@@ -19,6 +19,7 @@ package org.apache.nifi;
 import org.apache.nifi.bundle.Bundle;
 import org.apache.nifi.diagnostics.DiagnosticsFactory;
 import org.apache.nifi.nar.ExtensionMapping;
+import org.apache.nifi.util.NiFiProperties;
 
 import java.util.Set;
 
@@ -29,9 +30,7 @@ public interface NiFiServer {
 
     void start();
 
-    void setExtensionMapping(ExtensionMapping extensionMapping);
-
-    void setBundles(Bundle systemBundle, Set<Bundle> bundles);
+    void initialize(NiFiProperties properties, Bundle systemBundle, Set<Bundle> bundles, ExtensionMapping extensionMapping);
 
     void stop();
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/ExtensionMapping.java b/nifi-server-api/src/main/java/org/apache/nifi/nar/ExtensionMapping.java
similarity index 100%
rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/ExtensionMapping.java
rename to nifi-server-api/src/main/java/org/apache/nifi/nar/ExtensionMapping.java
diff --git a/pom.xml b/pom.xml
index fe7e57a..0a07556 100644
--- a/pom.xml
+++ b/pom.xml
@@ -25,6 +25,7 @@
         <module>nifi-commons</module>
         <module>nifi-api</module>
         <module>nifi-framework-api</module>
+        <module>nifi-server-api</module>
         <module>nifi-bootstrap</module>
         <module>nifi-mock</module>
         <module>nifi-nar-bundles</module>