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 0000000000..8086d780ac
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-headless-server/pom.xml
@@ -0,0 +1,90 @@
+
+
+
+ 4.0.0
+
+ org.apache.nifi
+ nifi-framework
+ 1.12.0-SNAPSHOT
+
+ nifi-headless-server
+ jar
+
+
+ org.slf4j
+ slf4j-log4j12
+
+
+ org.apache.nifi
+ nifi-api
+ compile
+
+
+ org.apache.nifi
+ nifi-runtime
+ compile
+
+
+ org.apache.nifi
+ nifi-framework-nar-loading-utils
+ compile
+
+
+ org.apache.nifi
+ nifi-nar-utils
+ compile
+
+
+ org.apache.nifi
+ nifi-properties
+ compile
+
+
+ org.apache.nifi
+ nifi-framework-core
+ compile
+
+
+ org.apache.nifi
+ nifi-framework-cluster
+ compile
+
+
+ org.apache.nifi
+ nifi-file-authorizer
+ compile
+
+
+ org.apache.commons
+ commons-text
+ 1.8
+
+
+ com.github.stefanbirkner
+ system-rules
+ 1.19.0
+ test
+
+
+ org.springframework
+ spring-test
+ 5.0.9.RELEASE
+ test
+
+
+
+
+
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 0000000000..c3503ee377
--- /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
+ *
+ * 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.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.
+ *
+ * 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 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 elementIdToMetadataMap = mapComponents(componentNodeList);
+
+ componentEnrichingMap.putAll(elementIdToMetadataMap);
+
+ }
+
+ // For each of the components we have, evaluate its dependencies and apply versions
+ for (Map.Entry 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 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 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 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.
+ *
+ * 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 mapComponents(NodeList parentNodes) {
+ final Map 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 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 0000000000..341345d4f0
--- /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,199 @@
+/*
+ * 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.DiagnosticsFactory;
+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.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 bundles;
+ private FlowService flowService;
+
+ 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);
+
+ // 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 bundles, ExtensionMapping extensionMapping) {
+ this.props = properties;
+ this.systemBundle = systemBundle;
+ this.bundles = bundles;
+ }
+
+ public DiagnosticsFactory getDiagnosticsFactory() {
+ return null;
+ }
+
+ public DiagnosticsFactory getThreadDumpFactory() {
+ return null;
+ }
+
+ 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 getBundles(final String bundleClass) {
+ return ExtensionManagerHolder.getExtensionManager().getBundles(bundleClass);
+ }
+}
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 0000000000..f317186081
--- /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 6258a1814f..37c5d1f843 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 @@
org.apache.nifi
nifi-framework-api
+
+ org.apache.nifi
+ nifi-server-api
+
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 380ade06cb..cd8fa460ba 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 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 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 narDetails = new ArrayList<>();
final Map 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 niFiServers = new HashMap<>();
int narCount;
do {
// record the number of nars to be loaded
@@ -224,7 +228,7 @@ public final class NarClassLoaders {
// see if this class loader is eligible for loading
ClassLoader narClassLoader = null;
if (narDependencyCoordinate == null) {
- narClassLoader = createNarClassLoader(narDetail.getWorkingDirectory(), jettyClassLoader);
+ narClassLoader = createNarClassLoader(narDetail.getWorkingDirectory(), rootClassloader);
} else {
final String dependencyCoordinateStr = narDependencyCoordinate.getCoordinate();
@@ -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 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 = 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 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);
}
@@ -501,6 +526,19 @@ public final class NarClassLoaders {
return initContext.jettyBundle;
}
+ /**
+ * @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
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 bb87ec37af..9fa92bbb84 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 d8823a8395..2184454227 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
@@ -27,6 +27,11 @@
nifi-framework-api
compile
+
+ org.apache.nifi
+ nifi-server-api
+ compile
+
org.apache.nifi
nifi-nar-utils
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 c0dbcbc99f..56394d07bd 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 acfed3727c..dc4d1c127c 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;
@@ -55,8 +54,9 @@ import java.util.concurrent.ThreadFactory;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
+import java.util.stream.Collectors;
-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 +142,17 @@ public class NiFi {
final Set 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.stream().filter((b) -> !narClassLoaders.getFrameworkBundle().equals(b)).collect(Collectors.toSet()),
+ extensionMapping);
if (shutdown) {
LOGGER.info("NiFi has been shutdown via NiFi Bootstrap. Will not start Controller");
@@ -167,7 +169,7 @@ public class NiFi {
}
}
- NiFiServer getServer() {
+ public NiFiServer getServer() {
return nifiServer;
}
@@ -186,7 +188,7 @@ public class NiFi {
@Override
public void run() {
// shutdown the jetty server
- shutdownHook();
+ shutdownHook(false);
}
}));
}
@@ -214,7 +216,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/NiFiEntryPoint.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/main/java/org/apache/nifi/NiFiEntryPoint.java
new file mode 100644
index 0000000000..386733325e
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/main/java/org/apache/nifi/NiFiEntryPoint.java
@@ -0,0 +1,24 @@
+/*
+ * 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;
+
+public interface NiFiEntryPoint {
+
+ NiFiServer getServer();
+
+ void shutdownHook(boolean isReload);
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/main/java/org/apache/nifi/headless/FlowEnrichmentException.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/main/java/org/apache/nifi/headless/FlowEnrichmentException.java
new file mode 100644
index 0000000000..acbe91b913
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/main/java/org/apache/nifi/headless/FlowEnrichmentException.java
@@ -0,0 +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
+ *
+ * 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;
+
+public class FlowEnrichmentException extends Exception {
+ public FlowEnrichmentException() {
+ super();
+ }
+
+ public FlowEnrichmentException(String message) {
+ super(message);
+ }
+
+ public FlowEnrichmentException(String message, Throwable cause) {
+ super(message, cause);
+ }
+
+ public FlowEnrichmentException(Throwable cause) {
+ super(cause);
+ }
+
+ 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 16c7cdbb1e..e8aa9b515b 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
@@ -32,6 +32,11 @@
nifi-api
compile
+
+ org.apache.nifi
+ nifi-server-api
+ compile
+
org.apache.nifi
nifi-runtime
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 ca7944f0a2..e83ee30f89 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 bundles;
@@ -162,13 +159,18 @@ public class JettyServer implements NiFiServer, ExtensionUiLoader {
private DeploymentManager deploymentManager;
- public JettyServer(final NiFiProperties props, final Set 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 extensionUiContexts = extensionUiInfo.getWebAppContexts();
- extensionUiContexts.stream().forEach(c -> webAppContextHandlers.addHandler(c));
+ extensionUiContexts.forEach(webAppContextHandlers::addHandler);
final ClassLoader frameworkClassLoader = getClass().getClassLoader();
@@ -1261,14 +1263,13 @@ public class JettyServer implements NiFiServer, ExtensionUiLoader {
}
@Override
- public void setExtensionMapping(ExtensionMapping extensionMapping) {
- this.extensionMapping = extensionMapping;
- }
-
- @Override
- public void setBundles(Bundle systemBundle, Set bundles) {
+ public void initialize(NiFiProperties properties, Bundle systemBundle, Set 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 0000000000..d2a3e145e6
--- /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 64077a0e72..9c54d35386 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
@@ -235,8 +235,9 @@ class JettyServerGroovyTest extends GroovyTestCase {
})
// Act
- JettyServer jettyServer = new JettyServer(mockProps, [] as Set)
-
+ JettyServer jettyServer = new JettyServer()
+ jettyServer.initialize(mockProps, null, [] as Set, 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 46de87a408..c819bf6207 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
@@ -190,6 +190,11 @@
nifi-framework-api
provided
+
+ org.apache.nifi
+ nifi-server-api
+ provided
+
org.apache.nifi
nifi-nar-utils
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 83cd951282..26683b6a79 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
@@ -47,6 +47,11 @@
nifi-framework-api
provided
+
+ org.apache.nifi
+ nifi-server-api
+ provided
+
javax.servlet
javax.servlet-api
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 030fa06aa4..9d6495752b 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 @@
nifi-mock-authorizer
nifi-shell-authorizer
nifi-stateless
+ nifi-headless-server
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-headless-server-nar/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-headless-server-nar/pom.xml
new file mode 100644
index 0000000000..5a649c0bba
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-headless-server-nar/pom.xml
@@ -0,0 +1,58 @@
+
+
+ 4.0.0
+
+ org.apache.nifi
+ nifi-framework-bundle
+ 1.12.0-SNAPSHOT
+
+ nifi-headless-server-nar
+ nar
+ MiNiFi server NAR
+
+ true
+ true
+
+
+
+ org.apache.nifi
+ nifi-headless-server
+ 1.12.0-SNAPSHOT
+
+
+
+
+ org.apache.nifi
+ nifi-api
+ provided
+
+
+ org.apache.nifi
+ nifi-runtime
+ provided
+
+
+ org.apache.nifi
+ nifi-nar-utils
+ provided
+
+
+ org.apache.nifi
+ nifi-properties
+ provided
+
+
+
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 0000000000..cc6a8e28eb
--- /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 0000000000..38f3bded16
--- /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 0000000000..4b58227952
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-server-nar/pom.xml
@@ -0,0 +1,63 @@
+
+
+ 4.0.0
+
+ org.apache.nifi
+ nifi-framework-bundle
+ 1.12.0-SNAPSHOT
+
+ nifi-server-nar
+ nar
+ NiFi: Web/UI Nar
+
+ true
+ true
+
+
+
+ org.apache.nifi
+ nifi-jetty-bundle
+ 1.12.0-SNAPSHOT
+ nar
+
+
+ org.apache.nifi
+ nifi-jetty
+
+
+
+
+ org.apache.nifi
+ nifi-api
+ provided
+
+
+ org.apache.nifi
+ nifi-runtime
+ provided
+
+
+ org.apache.nifi
+ nifi-nar-utils
+ provided
+
+
+ org.apache.nifi
+ nifi-properties
+ provided
+
+
+
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 0000000000..c6530dd457
--- /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 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
+
+ Based on Underscore.js, copyright Jeremy Ashkenas,
+ DocumentCloud and Investigative Reporters & Editors
+
+ 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 0000000000..5a88b5dc2f
--- /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 a5c1445b79..e3cd2794ed 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/pom.xml
@@ -30,6 +30,8 @@
nifi-framework
nifi-framework-nar
+ nifi-server-nar
+ nifi-headless-server-nar
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 3090f7f833..0096741d4c 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
@@ -26,6 +26,12 @@
true
+
+ org.apache.nifi
+ nifi-jetty-bundle
+ 1.12.0-SNAPSHOT
+ nar
+
org.apache.nifi
nifi-processor-utils
diff --git a/nifi-nar-bundles/pom.xml b/nifi-nar-bundles/pom.xml
index 53e61a06f0..6cae071630 100755
--- a/nifi-nar-bundles/pom.xml
+++ b/nifi-nar-bundles/pom.xml
@@ -294,6 +294,12 @@
1.12.0-SNAPSHOT
provided
+
+ org.apache.nifi
+ nifi-server-api
+ 1.12.0-SNAPSHOT
+ provided
+
org.apache.nifi
nifi-runtime
diff --git a/nifi-server-api/pom.xml b/nifi-server-api/pom.xml
new file mode 100644
index 0000000000..f174f122b0
--- /dev/null
+++ b/nifi-server-api/pom.xml
@@ -0,0 +1,46 @@
+
+
+
+ 4.0.0
+
+ org.apache.nifi
+ nifi
+ 1.12.0-SNAPSHOT
+
+ nifi-server-api
+ jar
+
+
+
+ org.apache.nifi
+ nifi-api
+ 1.12.0-SNAPSHOT
+ provided
+
+
+ org.apache.nifi
+ nifi-framework-api
+ 1.12.0-SNAPSHOT
+ provided
+
+
+ org.apache.nifi
+ nifi-properties
+ 1.12.0-SNAPSHOT
+ provided
+
+
+
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 21695797bc..8124876975 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 bundles);
+ void initialize(NiFiProperties properties, Bundle systemBundle, Set 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 eab081c70f..7174f31509 100644
--- a/pom.xml
+++ b/pom.xml
@@ -25,6 +25,7 @@
nifi-commons
nifi-api
nifi-framework-api
+ nifi-server-api
nifi-bootstrap
nifi-mock
nifi-nar-bundles