You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by ma...@apache.org on 2017/03/24 15:10:55 UTC

[08/17] nifi git commit: NIFI-3380 Bumping NAR plugin to 1.2.0-SNAPSHOT development to leverage changes from master, adding buildnumber-maven-plugin to nifi-nar-bundles to properly set build info in MANIFEST of NARs - Refactoring NarDetails to include al

http://git-wip-us.apache.org/repos/asf/nifi/blob/d90cf846/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarClassLoaders.java
----------------------------------------------------------------------
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 9a566c3..a656e76 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,27 +16,29 @@
  */
 package org.apache.nifi.nar;
 
+import org.apache.nifi.bundle.Bundle;
+import org.apache.nifi.bundle.BundleCoordinate;
+import org.apache.nifi.bundle.BundleDetails;
+import org.apache.nifi.util.FileUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import java.io.File;
-import java.io.FileInputStream;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.Iterator;
 import java.util.LinkedHashMap;
 import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-import java.util.jar.Attributes;
-import java.util.jar.Manifest;
-import org.apache.nifi.util.FileUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+
 
 /**
- * A singleton class used to initialize the extension and framework
- * classloaders.
+ * A singleton class used to initialize the extension and framework classloaders.
  */
 public final class NarClassLoaders {
 
@@ -51,18 +53,18 @@ public final class NarClassLoaders {
 
         private final File frameworkWorkingDir;
         private final File extensionWorkingDir;
-        private final ClassLoader frameworkClassLoader;
-        private final Map<String, ClassLoader> extensionClassLoaders;
+        private final Bundle frameworkBundle;
+        private final Map<String, Bundle> bundles;
 
         private InitContext(
                 final File frameworkDir,
                 final File extensionDir,
-                final ClassLoader frameworkClassloader,
-                final Map<String, ClassLoader> extensionClassLoaders) {
+                final Bundle frameworkBundle,
+                final Map<String, Bundle> bundles) {
             this.frameworkWorkingDir = frameworkDir;
             this.extensionWorkingDir = extensionDir;
-            this.frameworkClassLoader = frameworkClassloader;
-            this.extensionClassLoaders = extensionClassLoaders;
+            this.frameworkBundle = frameworkBundle;
+            this.bundles = bundles;
         }
     }
 
@@ -126,12 +128,13 @@ public final class NarClassLoaders {
         final ClassLoader systemClassLoader = ClassLoader.getSystemClassLoader();
 
         // find all nar files and create class loaders for them.
-        final Map<String, ClassLoader> extensionDirectoryClassLoaderLookup = new LinkedHashMap<>();
-        final Map<String, ClassLoader> narIdClassLoaderLookup = new HashMap<>();
+        final Map<String, Bundle> narDirectoryBundleLookup = new LinkedHashMap<>();
+        final Map<String, ClassLoader> narCoordinateClassLoaderLookup = new HashMap<>();
+        final Map<String, Set<BundleCoordinate>> narIdBundleLookup = new HashMap<>();
 
         // make sure the nar directory is there and accessible
-        FileUtils.ensureDirectoryExistAndCanAccess(frameworkWorkingDir);
-        FileUtils.ensureDirectoryExistAndCanAccess(extensionsWorkingDir);
+        FileUtils.ensureDirectoryExistAndCanReadAndWrite(frameworkWorkingDir);
+        FileUtils.ensureDirectoryExistAndCanReadAndWrite(extensionsWorkingDir);
 
         final List<File> narWorkingDirContents = new ArrayList<>();
         final File[] frameworkWorkingDirContents = frameworkWorkingDir.listFiles();
@@ -144,37 +147,49 @@ public final class NarClassLoaders {
         }
 
         if (!narWorkingDirContents.isEmpty()) {
-            final List<NarDetails> narDetails = new ArrayList<>();
+            final List<BundleDetails> narDetails = new ArrayList<>();
+            final Map<String,String> narCoordinatesToWorkingDir = new HashMap<>();
 
             // load the nar details which includes and nar dependencies
             for (final File unpackedNar : narWorkingDirContents) {
-                final NarDetails narDetail = getNarDetails(unpackedNar);
+                BundleDetails narDetail = null;
+                try {
+                     narDetail = getNarDetails(unpackedNar);
+                } catch (IllegalStateException e) {
+                    logger.warn("Unable to load NAR {} due to {}, skipping...",
+                            new Object[] {unpackedNar.getAbsolutePath(), e.getMessage()});
+                }
 
-                // ensure the nar contained an identifier
-                if (narDetail.getNarId() == null) {
-                    logger.warn("No NAR Id found. Skipping: " + unpackedNar.getAbsolutePath());
-                    continue;
+                // prevent the application from starting when there are two NARs with same group, id, and version
+                final String narCoordinate = narDetail.getCoordinate().getCoordinate();
+                if (narCoordinatesToWorkingDir.containsKey(narCoordinate)) {
+                    final String existingNarWorkingDir = narCoordinatesToWorkingDir.get(narCoordinate);
+                    throw new IllegalStateException("Unable to load NAR with coordinates " + narCoordinate
+                            + " and working directory " + narDetail.getWorkingDirectory()
+                            + " because another NAR with the same coordinates already exists at " + existingNarWorkingDir);
                 }
 
-                // store the nar details
                 narDetails.add(narDetail);
+                narCoordinatesToWorkingDir.put(narCoordinate, narDetail.getWorkingDirectory().getCanonicalPath());
             }
 
             // attempt to locate the jetty nar
             ClassLoader jettyClassLoader = null;
-            for (final Iterator<NarDetails> narDetailsIter = narDetails.iterator(); narDetailsIter.hasNext();) {
-                final NarDetails narDetail = narDetailsIter.next();
+            for (final Iterator<BundleDetails> narDetailsIter = narDetails.iterator(); narDetailsIter.hasNext();) {
+                final BundleDetails narDetail = narDetailsIter.next();
 
                 // look for the jetty nar
-                if (JETTY_NAR_ID.equals(narDetail.getNarId())) {
+                if (JETTY_NAR_ID.equals(narDetail.getCoordinate().getId())) {
                     // create the jetty classloader
-                    jettyClassLoader = createNarClassLoader(narDetail.getNarWorkingDirectory(), systemClassLoader);
+                    jettyClassLoader = createNarClassLoader(narDetail.getWorkingDirectory(), systemClassLoader);
 
                     // remove the jetty nar since its already loaded
-                    narIdClassLoaderLookup.put(narDetail.getNarId(), jettyClassLoader);
+                    narCoordinateClassLoaderLookup.put(narDetail.getCoordinate().getCoordinate(), jettyClassLoader);
                     narDetailsIter.remove();
-                    break;
                 }
+
+                // populate bundle lookup
+                narIdBundleLookup.computeIfAbsent(narDetail.getCoordinate().getId(), id -> new HashSet<>()).add(narDetail.getCoordinate());
             }
 
             // ensure the jetty nar was found
@@ -188,22 +203,50 @@ public final class NarClassLoaders {
                 narCount = narDetails.size();
 
                 // attempt to create each nar class loader
-                for (final Iterator<NarDetails> narDetailsIter = narDetails.iterator(); narDetailsIter.hasNext();) {
-                    final NarDetails narDetail = narDetailsIter.next();
-                    final String narDependencies = narDetail.getNarDependencyId();
+                for (final Iterator<BundleDetails> narDetailsIter = narDetails.iterator(); narDetailsIter.hasNext();) {
+                    final BundleDetails narDetail = narDetailsIter.next();
+                    final BundleCoordinate narDependencyCoordinate = narDetail.getDependencyCoordinate();
 
                     // see if this class loader is eligible for loading
                     ClassLoader narClassLoader = null;
-                    if (narDependencies == null) {
-                        narClassLoader = createNarClassLoader(narDetail.getNarWorkingDirectory(), jettyClassLoader);
-                    } else if (narIdClassLoaderLookup.containsKey(narDetail.getNarDependencyId())) {
-                        narClassLoader = createNarClassLoader(narDetail.getNarWorkingDirectory(), narIdClassLoaderLookup.get(narDetail.getNarDependencyId()));
+                    if (narDependencyCoordinate == null) {
+                        narClassLoader = createNarClassLoader(narDetail.getWorkingDirectory(), jettyClassLoader);
+                    } else {
+                        final String dependencyCoordinateStr = narDependencyCoordinate.getCoordinate();
+
+                        // if the declared dependency has already been loaded
+                        if (narCoordinateClassLoaderLookup.containsKey(dependencyCoordinateStr)) {
+                            final ClassLoader narDependencyClassLoader = narCoordinateClassLoaderLookup.get(dependencyCoordinateStr);
+                            narClassLoader = createNarClassLoader(narDetail.getWorkingDirectory(), narDependencyClassLoader);
+                        } else {
+                            // get all bundles that match the declared dependency id
+                            final Set<BundleCoordinate> coordinates = narIdBundleLookup.get(narDependencyCoordinate.getId());
+
+                            // ensure there are known bundles that match the declared dependency id
+                            if (coordinates != null && !coordinates.contains(narDependencyCoordinate)) {
+                                // ensure the declared dependency only has one possible bundle
+                                if (coordinates.size() == 1) {
+                                    // get the bundle with the matching id
+                                    final BundleCoordinate coordinate = coordinates.stream().findFirst().get();
+
+                                    // if that bundle is loaded, use it
+                                    if (narCoordinateClassLoaderLookup.containsKey(coordinate.getCoordinate())) {
+                                        logger.warn(String.format("While loading '%s' unable to locate exact NAR dependency '%s'. Only found one possible match '%s'. Continuing...",
+                                                narDetail.getCoordinate().getCoordinate(), dependencyCoordinateStr, coordinate.getCoordinate()));
+
+                                        final ClassLoader narDependencyClassLoader = narCoordinateClassLoaderLookup.get(coordinate.getCoordinate());
+                                        narClassLoader = createNarClassLoader(narDetail.getWorkingDirectory(), narDependencyClassLoader);
+                                    }
+                                }
+                            }
+                        }
                     }
 
                     // if we were able to create the nar class loader, store it and remove the details
-                    if (narClassLoader != null) {
-                        extensionDirectoryClassLoaderLookup.put(narDetail.getNarWorkingDirectory().getCanonicalPath(), narClassLoader);
-                        narIdClassLoaderLookup.put(narDetail.getNarId(), narClassLoader);
+                    final ClassLoader bundleClassLoader = narClassLoader;
+                    if (bundleClassLoader != null) {
+                        narDirectoryBundleLookup.put(narDetail.getWorkingDirectory().getCanonicalPath(), new Bundle(narDetail, bundleClassLoader));
+                        narCoordinateClassLoaderLookup.put(narDetail.getCoordinate().getCoordinate(), narClassLoader);
                         narDetailsIter.remove();
                     }
                 }
@@ -212,12 +255,18 @@ public final class NarClassLoaders {
             } while (narCount != narDetails.size());
 
             // see if any nars couldn't be loaded
-            for (final NarDetails narDetail : narDetails) {
-                logger.warn(String.format("Unable to resolve required dependency '%s'. Skipping NAR %s", narDetail.getNarDependencyId(), narDetail.getNarWorkingDirectory().getAbsolutePath()));
+            for (final BundleDetails narDetail : narDetails) {
+                logger.warn(String.format("Unable to resolve required dependency '%s'. Skipping NAR '%s'",
+                        narDetail.getDependencyCoordinate().getId(), narDetail.getWorkingDirectory().getAbsolutePath()));
             }
         }
 
-        return new InitContext(frameworkWorkingDir, extensionsWorkingDir, narIdClassLoaderLookup.get(FRAMEWORK_NAR_ID), new LinkedHashMap<>(extensionDirectoryClassLoaderLookup));
+        // find the framework bundle, NarUnpacker already checked that there was a framework NAR and that there was only one
+        final Bundle frameworkBundle = narDirectoryBundleLookup.values().stream()
+                .filter(b -> b.getBundleDetails().getCoordinate().getId().equals(FRAMEWORK_NAR_ID))
+                .findFirst().orElse(null);
+
+        return new InitContext(frameworkWorkingDir, extensionsWorkingDir, frameworkBundle, new LinkedHashMap<>(narDirectoryBundleLookup));
     }
 
     /**
@@ -244,50 +293,36 @@ public final class NarClassLoaders {
      * @return details about the NAR
      * @throws IOException ioe
      */
-    private static NarDetails getNarDetails(final File narDirectory) throws IOException {
-        final NarDetails narDetails = new NarDetails();
-        narDetails.setNarWorkingDirectory(narDirectory);
-
-        final File manifestFile = new File(narDirectory, "META-INF/MANIFEST.MF");
-        try (final FileInputStream fis = new FileInputStream(manifestFile)) {
-            final Manifest manifest = new Manifest(fis);
-            final Attributes attributes = manifest.getMainAttributes();
-
-            // get the nar details
-            narDetails.setNarId(attributes.getValue("Nar-Id"));
-            narDetails.setNarDependencyId(attributes.getValue("Nar-Dependency-Id"));
-        }
-
-        return narDetails;
+    private static BundleDetails getNarDetails(final File narDirectory) throws IOException {
+        return NarBundleUtil.fromNarDirectory(narDirectory);
     }
 
     /**
-     * @return the framework class loader
+     * @return the framework class Bundle
      *
-     * @throws IllegalStateException if the frame class loader has not been
-     * loaded
+     * @throws IllegalStateException if the frame Bundle has not been loaded
      */
-    public ClassLoader getFrameworkClassLoader() {
+    public Bundle getFrameworkBundle() {
         if (initContext == null) {
-            throw new IllegalStateException("Framework class loader has not been loaded.");
+            throw new IllegalStateException("Framework bundle has not been loaded.");
         }
 
-        return initContext.frameworkClassLoader;
+        return initContext.frameworkBundle;
     }
 
     /**
      * @param extensionWorkingDirectory the directory
-     * @return the class loader for the specified working directory. Returns
-     * null when no class loader exists for the specified working directory
-     * @throws IllegalStateException if the class loaders have not been loaded
+     * @return the bundle for the specified working directory. Returns
+     * null when no bundle exists for the specified working directory
+     * @throws IllegalStateException if the bundles have not been loaded
      */
-    public ClassLoader getExtensionClassLoader(final File extensionWorkingDirectory) {
+    public Bundle getBundle(final File extensionWorkingDirectory) {
         if (initContext == null) {
             throw new IllegalStateException("Extensions class loaders have not been loaded.");
         }
 
         try {
-            return initContext.extensionClassLoaders.get(extensionWorkingDirectory.getCanonicalPath());
+           return initContext.bundles.get(extensionWorkingDirectory.getCanonicalPath());
         } catch (final IOException ioe) {
             if(logger.isDebugEnabled()){
                 logger.debug("Unable to get extension classloader for working directory '{}'", extensionWorkingDirectory);
@@ -297,46 +332,15 @@ public final class NarClassLoaders {
     }
 
     /**
-     * @return the extension class loaders
-     * @throws IllegalStateException if the class loaders have not been loaded
+     * @return the extensions that have been loaded
+     * @throws IllegalStateException if the extensions have not been loaded
      */
-    public Set<ClassLoader> getExtensionClassLoaders() {
+    public Set<Bundle> getBundles() {
         if (initContext == null) {
-            throw new IllegalStateException("Extensions class loaders have not been loaded.");
+            throw new IllegalStateException("Bundles have not been loaded.");
         }
 
-        return new LinkedHashSet<>(initContext.extensionClassLoaders.values());
-    }
-
-    private static class NarDetails {
-
-        private String narId;
-        private String narDependencyId;
-        private File narWorkingDirectory;
-
-        public String getNarDependencyId() {
-            return narDependencyId;
-        }
-
-        public void setNarDependencyId(String narDependencyId) {
-            this.narDependencyId = narDependencyId;
-        }
-
-        public String getNarId() {
-            return narId;
-        }
-
-        public void setNarId(String narId) {
-            this.narId = narId;
-        }
-
-        public File getNarWorkingDirectory() {
-            return narWorkingDirectory;
-        }
-
-        public void setNarWorkingDirectory(File narWorkingDirectory) {
-            this.narWorkingDirectory = narWorkingDirectory;
-        }
+        return new LinkedHashSet<>(initContext.bundles.values());
     }
 
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/d90cf846/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarCloseable.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarCloseable.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarCloseable.java
index 9bcdb05..88d47ff 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarCloseable.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarCloseable.java
@@ -48,7 +48,7 @@ public class NarCloseable implements Closeable {
     public static NarCloseable withComponentNarLoader(final Class componentClass, final String componentIdentifier) {
         final ClassLoader current = Thread.currentThread().getContextClassLoader();
 
-        ClassLoader componentClassLoader = ExtensionManager.getClassLoader(componentClass.getName(), componentIdentifier);
+        ClassLoader componentClassLoader = ExtensionManager.getInstanceClassLoader(componentIdentifier);
         if (componentClassLoader == null) {
             componentClassLoader = componentClass.getClassLoader();
         }
@@ -58,6 +58,20 @@ public class NarCloseable implements Closeable {
     }
 
     /**
+     * Sets the current thread context class loader to the provided class loader, and returns a NarCloseable that will
+     * return the current thread context class loader to it's previous state.
+     *
+     * @param componentNarLoader the class loader to set as the current thread context class loader
+     *
+     * @return NarCloseable that will return the current thread context class loader to its previous state
+     */
+    public static NarCloseable withComponentNarLoader(final ClassLoader componentNarLoader) {
+        final ClassLoader current = Thread.currentThread().getContextClassLoader();
+        Thread.currentThread().setContextClassLoader(componentNarLoader);
+        return new NarCloseable(current);
+    }
+
+    /**
      * Creates a Closeable object that can be used to to switch to current class
      * loader to the framework class loader and will automatically set the
      * ClassLoader back to the previous class loader when closed
@@ -67,7 +81,7 @@ public class NarCloseable implements Closeable {
     public static NarCloseable withFrameworkNar() {
         final ClassLoader frameworkClassLoader;
         try {
-            frameworkClassLoader = NarClassLoaders.getInstance().getFrameworkClassLoader();
+            frameworkClassLoader = NarClassLoaders.getInstance().getFrameworkBundle().getClassLoader();
         } catch (final Exception e) {
             // This should never happen in a running instance, but it will occur in unit tests
             logger.error("Unable to access Framework ClassLoader due to " + e + ". Will continue without changing ClassLoaders.");

http://git-wip-us.apache.org/repos/asf/nifi/blob/d90cf846/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarManifestEntry.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarManifestEntry.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarManifestEntry.java
new file mode 100644
index 0000000..da54c4e
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarManifestEntry.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.nar;
+
+/**
+ * Enumeration of entries that will be in a NAR MANIFEST file.
+ */
+public enum NarManifestEntry {
+
+    NAR_GROUP("Nar-Group"),
+    NAR_ID("Nar-Id"),
+    NAR_VERSION("Nar-Version"),
+    NAR_DEPENDENCY_GROUP("Nar-Dependency-Group"),
+    NAR_DEPENDENCY_ID("Nar-Dependency-Id"),
+    NAR_DEPENDENCY_VERSION("Nar-Dependency-Version"),
+    BUILD_TAG("Build-Tag"),
+    BUILD_REVISION("Build-Revision"),
+    BUILD_BRANCH("Build-Branch"),
+    BUILD_TIMESTAMP("Build-Timestamp"),
+    BUILD_JDK("Build-Jdk"),
+    BUILT_BY("Built-By")
+    ;
+
+    final String manifestName;
+
+    NarManifestEntry(String manifestName) {
+        this.manifestName = manifestName;
+    }
+
+    public String getManifestName() {
+        return manifestName;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/d90cf846/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarThreadContextClassLoader.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarThreadContextClassLoader.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarThreadContextClassLoader.java
index 3244456..e7faa02 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarThreadContextClassLoader.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarThreadContextClassLoader.java
@@ -18,7 +18,9 @@ package org.apache.nifi.nar;
 
 import org.apache.nifi.authentication.LoginIdentityProvider;
 import org.apache.nifi.authorization.Authorizer;
+import org.apache.nifi.bundle.Bundle;
 import org.apache.nifi.components.Validator;
+import org.apache.nifi.components.state.StateProvider;
 import org.apache.nifi.controller.ControllerService;
 import org.apache.nifi.controller.repository.ContentRepository;
 import org.apache.nifi.controller.repository.FlowFileRepository;
@@ -31,6 +33,7 @@ import org.apache.nifi.processor.io.OutputStreamCallback;
 import org.apache.nifi.processor.io.StreamCallback;
 import org.apache.nifi.provenance.ProvenanceRepository;
 import org.apache.nifi.reporting.ReportingTask;
+import org.apache.nifi.util.NiFiProperties;
 
 import java.io.IOException;
 import java.io.InputStream;
@@ -41,7 +44,6 @@ import java.net.URLClassLoader;
 import java.util.ArrayList;
 import java.util.Enumeration;
 import java.util.List;
-import org.apache.nifi.util.NiFiProperties;
 
 /**
  * THREAD SAFE
@@ -68,6 +70,7 @@ public class NarThreadContextClassLoader extends URLClassLoader {
         narSpecificClasses.add(FlowFileRepository.class);
         narSpecificClasses.add(FlowFileSwapManager.class);
         narSpecificClasses.add(ContentRepository.class);
+        narSpecificClasses.add(StateProvider.class);
     }
 
     private NarThreadContextClassLoader() {
@@ -187,15 +190,17 @@ public class NarThreadContextClassLoader extends URLClassLoader {
         final ClassLoader originalClassLoader = Thread.currentThread().getContextClassLoader();
         Thread.currentThread().setContextClassLoader(NarThreadContextClassLoader.getInstance());
         try {
-            final ClassLoader detectedClassLoaderForType = ExtensionManager.getClassLoader(implementationClassName);
-            final Class<?> rawClass;
-            if (detectedClassLoaderForType == null) {
-                // try to find from the current class loader
-                rawClass = Class.forName(implementationClassName);
-            } else {
-                // try to find from the registered classloader for that type
-                rawClass = Class.forName(implementationClassName, true, ExtensionManager.getClassLoader(implementationClassName));
+            final List<Bundle> bundles = ExtensionManager.getBundles(implementationClassName);
+            if (bundles.size() == 0) {
+                throw new IllegalStateException(String.format("The specified implementation class '%s' is not known to this nifi.", implementationClassName));
             }
+            if (bundles.size() > 1) {
+                throw new IllegalStateException(String.format("More than one bundle was found for the specified implementation class '%s', only one is allowed.", implementationClassName));
+            }
+
+            final Bundle bundle = bundles.get(0);
+            final ClassLoader detectedClassLoaderForType = bundle.getClassLoader();
+            final Class<?> rawClass = Class.forName(implementationClassName, true, detectedClassLoaderForType);
 
             Thread.currentThread().setContextClassLoader(detectedClassLoaderForType);
             final Class<?> desiredClass = rawClass.asSubclass(typeDefinition);

http://git-wip-us.apache.org/repos/asf/nifi/blob/d90cf846/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarUnpacker.java
----------------------------------------------------------------------
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 1e9ac47..fdaa017 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
@@ -16,6 +16,14 @@
  */
 package org.apache.nifi.nar;
 
+import org.apache.nifi.bundle.Bundle;
+import org.apache.nifi.bundle.BundleCoordinate;
+import org.apache.nifi.util.FileUtils;
+import org.apache.nifi.util.NiFiProperties;
+import org.apache.nifi.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import java.io.BufferedReader;
 import java.io.File;
 import java.io.FileFilter;
@@ -31,20 +39,16 @@ import java.security.NoSuchAlgorithmException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Enumeration;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 import java.util.jar.Attributes;
 import java.util.jar.JarEntry;
 import java.util.jar.JarFile;
 import java.util.jar.Manifest;
 
-import org.apache.nifi.util.FileUtils;
-import org.apache.nifi.util.NiFiProperties;
-import org.apache.nifi.util.StringUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 /**
  *
  */
@@ -60,11 +64,12 @@ public final class NarUnpacker {
         }
     };
 
-    public static ExtensionMapping unpackNars(final NiFiProperties props) {
+    public static ExtensionMapping unpackNars(final NiFiProperties props, final Bundle systemBundle) {
         final List<Path> narLibraryDirs = props.getNarLibraryDirectories();
         final File frameworkWorkingDir = props.getFrameworkWorkingDirectory();
         final File extensionsWorkingDir = props.getExtensionsWorkingDirectory();
         final File docsWorkingDir = props.getComponentDocumentationWorkingDirectory();
+        final Map<File, BundleCoordinate> unpackedNars = new HashMap<>();
 
         try {
             File unpackedFramework = null;
@@ -102,18 +107,26 @@ public final class NarUnpacker {
 
                         // lookup the nar id
                         final Attributes attributes = manifest.getMainAttributes();
-                        final String narId = attributes.getValue("Nar-Id");
+                        final String groupId = attributes.getValue(NarManifestEntry.NAR_GROUP.getManifestName());
+                        final String narId = attributes.getValue(NarManifestEntry.NAR_ID.getManifestName());
+                        final String version = attributes.getValue(NarManifestEntry.NAR_VERSION.getManifestName());
 
                         // determine if this is the framework
                         if (NarClassLoaders.FRAMEWORK_NAR_ID.equals(narId)) {
                             if (unpackedFramework != null) {
-                                throw new IllegalStateException(
-                                        "Multiple framework NARs discovered. Only one framework is permitted.");
+                                throw new IllegalStateException("Multiple framework NARs discovered. Only one framework is permitted.");
                             }
 
+                            // unpack the framework nar
                             unpackedFramework = unpackNar(narFile, frameworkWorkingDir);
                         } else {
-                            unpackedExtensions.add(unpackNar(narFile, extensionsWorkingDir));
+                            final File unpackedExtension = unpackNar(narFile, extensionsWorkingDir);
+
+                            // record the current bundle
+                            unpackedNars.put(unpackedExtension, new BundleCoordinate(groupId, narId, version));
+
+                            // unpack the extension nar
+                            unpackedExtensions.add(unpackedExtension);
                         }
                     }
                 }
@@ -125,10 +138,8 @@ public final class NarUnpacker {
                     throw new IllegalStateException("Framework NAR cannot be read.");
                 }
 
-                // Determine if any nars no longer exist and delete their
-                // working directories. This happens
-                // if a new version of a nar is dropped into the lib dir.
-                // ensure no old framework are present
+                // Determine if any nars no longer exist and delete their working directories. This happens
+                // if a new version of a nar is dropped into the lib dir. ensure no old framework are present
                 final File[] frameworkWorkingDirContents = frameworkWorkingDir.listFiles();
                 if (frameworkWorkingDirContents != null) {
                     for (final File unpackedNar : frameworkWorkingDirContents) {
@@ -151,8 +162,7 @@ public final class NarUnpacker {
                 logger.info("NAR loading process took " + (endTime - startTime) + " nanoseconds.");
             }
 
-            // attempt to delete any docs files that exist so that any
-            // components that have been removed
+            // attempt to delete any docs files that exist so that any components that have been removed
             // will no longer have entries in the docs folder
             final File[] docsFiles = docsWorkingDir.listFiles();
             if (docsFiles != null) {
@@ -162,11 +172,14 @@ public final class NarUnpacker {
             }
 
             final ExtensionMapping extensionMapping = new ExtensionMapping();
-            mapExtensions(extensionsWorkingDir, docsWorkingDir, extensionMapping);
+            mapExtensions(unpackedNars, docsWorkingDir, extensionMapping);
+
+            // unpack docs for the system bundle which will catch any JARs directly in the lib directory that might have docs
+            unpackBundleDocs(docsWorkingDir, extensionMapping, systemBundle.getBundleDetails().getCoordinate(), systemBundle.getBundleDetails().getWorkingDirectory());
+
             return extensionMapping;
         } catch (IOException e) {
-            logger.warn("Unable to load NAR library bundles due to " + e
-                    + " Will proceed without loading any further Nar bundles");
+            logger.warn("Unable to load NAR library bundles due to " + e + " Will proceed without loading any further Nar bundles");
             if (logger.isDebugEnabled()) {
                 logger.warn("", e);
             }
@@ -175,15 +188,23 @@ public final class NarUnpacker {
         return null;
     }
 
-    private static void mapExtensions(final File workingDirectory, final File docsDirectory,
-            final ExtensionMapping mapping) throws IOException {
-        final File[] directoryContents = workingDirectory.listFiles();
+    private static void mapExtensions(final Map<File, BundleCoordinate> unpackedNars, final File docsDirectory, final ExtensionMapping mapping) throws IOException {
+        for (final Map.Entry<File, BundleCoordinate> entry : unpackedNars.entrySet()) {
+            final File unpackedNar = entry.getKey();
+            final BundleCoordinate bundleCoordinate = entry.getValue();
+
+            final File bundledDependencies = new File(unpackedNar, "META-INF/bundled-dependencies");
+
+            unpackBundleDocs(docsDirectory, mapping, bundleCoordinate, bundledDependencies);
+        }
+    }
+
+    private static void unpackBundleDocs(final File docsDirectory, final ExtensionMapping mapping, final BundleCoordinate bundleCoordinate, final File bundledDirectory) throws IOException {
+        final File[] directoryContents = bundledDirectory.listFiles();
         if (directoryContents != null) {
             for (final File file : directoryContents) {
-                if (file.isDirectory()) {
-                    mapExtensions(file, docsDirectory, mapping);
-                } else if (file.getName().toLowerCase().endsWith(".jar")) {
-                    unpackDocumentation(file, docsDirectory, mapping);
+                if (file.getName().toLowerCase().endsWith(".jar")) {
+                    unpackDocumentation(bundleCoordinate, file, docsDirectory, mapping);
                 }
             }
         }
@@ -192,16 +213,12 @@ public final class NarUnpacker {
     /**
      * Unpacks the specified nar into the specified base working directory.
      *
-     * @param nar
-     *            the nar to unpack
-     * @param baseWorkingDirectory
-     *            the directory to unpack to
+     * @param nar the nar to unpack
+     * @param baseWorkingDirectory the directory to unpack to
      * @return the directory to the unpacked NAR
-     * @throws IOException
-     *             if unable to explode nar
+     * @throws IOException if unable to explode nar
      */
-    private static File unpackNar(final File nar, final File baseWorkingDirectory)
-            throws IOException {
+    private static File unpackNar(final File nar, final File baseWorkingDirectory) throws IOException {
         final File narWorkingDirectory = new File(baseWorkingDirectory, nar.getName() + "-unpacked");
 
         // if the working directory doesn't exist, unpack the nar
@@ -218,8 +235,7 @@ public final class NarUnpacker {
             } else {
                 final byte[] hashFileContents = Files.readAllBytes(workingHashFile.toPath());
                 if (!Arrays.equals(hashFileContents, narMd5)) {
-                    logger.info("Contents of nar {} have changed. Reloading.",
-                            new Object[] { nar.getAbsolutePath() });
+                    logger.info("Contents of nar {} have changed. Reloading.", new Object[] { nar.getAbsolutePath() });
                     FileUtils.deleteFile(narWorkingDirectory, true);
                     unpack(nar, narWorkingDirectory, narMd5);
                 }
@@ -233,13 +249,10 @@ public final class NarUnpacker {
      * Unpacks the NAR to the specified directory. Creates a checksum file that
      * used to determine if future expansion is necessary.
      *
-     * @param workingDirectory
-     *            the root directory to which the NAR should be unpacked.
-     * @throws IOException
-     *             if the NAR could not be unpacked.
+     * @param workingDirectory the root directory to which the NAR should be unpacked.
+     * @throws IOException if the NAR could not be unpacked.
      */
-    private static void unpack(final File nar, final File workingDirectory, final byte[] hash)
-            throws IOException {
+    private static void unpack(final File nar, final File workingDirectory, final byte[] hash) throws IOException {
 
         try (JarFile jarFile = new JarFile(nar)) {
             Enumeration<JarEntry> jarEntries = jarFile.entries();
@@ -261,41 +274,43 @@ public final class NarUnpacker {
         }
     }
 
-    private static void unpackDocumentation(final File jar, final File docsDirectory,
-            final ExtensionMapping extensionMapping) throws IOException {
-        // determine the components that may have documentation
-        if (!determineDocumentedNiFiComponents(jar, extensionMapping)) {
+    private static void unpackDocumentation(final BundleCoordinate coordinate, final File jar, final File docsDirectory, final ExtensionMapping extensionMapping) throws IOException {
+        final ExtensionMapping jarExtensionMapping = determineDocumentedNiFiComponents(coordinate, jar);
+
+        // skip if there are not components to document
+        if (jarExtensionMapping.isEmpty()) {
             return;
         }
 
+        // merge the extension mapping found in this jar
+        extensionMapping.merge(jarExtensionMapping);
+
         // look for all documentation related to each component
         try (final JarFile jarFile = new JarFile(jar)) {
-            for (final String componentName : extensionMapping.getAllExtensionNames()) {
+            for (final String componentName : jarExtensionMapping.getAllExtensionNames().keySet()) {
                 final String entryName = "docs/" + componentName;
 
                 // go through each entry in this jar
-                for (final Enumeration<JarEntry> jarEnumeration = jarFile.entries(); jarEnumeration
-                        .hasMoreElements();) {
+                for (final Enumeration<JarEntry> jarEnumeration = jarFile.entries(); jarEnumeration.hasMoreElements();) {
                     final JarEntry jarEntry = jarEnumeration.nextElement();
 
                     // if this entry is documentation for this component
                     if (jarEntry.getName().startsWith(entryName)) {
                         final String name = StringUtils.substringAfter(jarEntry.getName(), "docs/");
+                        final String path = coordinate.getGroup() + "/" + coordinate.getId() + "/" + coordinate.getVersion() + "/" + name;
 
                         // if this is a directory create it
                         if (jarEntry.isDirectory()) {
-                            final File componentDocsDirectory = new File(docsDirectory, name);
+                            final File componentDocsDirectory = new File(docsDirectory, path);
 
                             // ensure the documentation directory can be created
-                            if (!componentDocsDirectory.exists()
-                                    && !componentDocsDirectory.mkdirs()) {
-                                logger.warn("Unable to create docs directory "
-                                        + componentDocsDirectory.getAbsolutePath());
+                            if (!componentDocsDirectory.exists() && !componentDocsDirectory.mkdirs()) {
+                                logger.warn("Unable to create docs directory " + componentDocsDirectory.getAbsolutePath());
                                 break;
                             }
                         } else {
                             // if this is a file, write to it
-                            final File componentDoc = new File(docsDirectory, name);
+                            final File componentDoc = new File(docsDirectory, path);
                             makeFile(jarFile.getInputStream(jarEntry), componentDoc);
                         }
                     }
@@ -308,32 +323,26 @@ public final class NarUnpacker {
     /*
      * Returns true if this jar file contains a NiFi component
      */
-    private static boolean determineDocumentedNiFiComponents(final File jar,
-            final ExtensionMapping extensionMapping) throws IOException {
+    private static ExtensionMapping determineDocumentedNiFiComponents(final BundleCoordinate coordinate, final File jar) throws IOException {
+        final ExtensionMapping mapping = new ExtensionMapping();
+
         try (final JarFile jarFile = new JarFile(jar)) {
-            final JarEntry processorEntry = jarFile
-                    .getJarEntry("META-INF/services/org.apache.nifi.processor.Processor");
-            final JarEntry reportingTaskEntry = jarFile
-                    .getJarEntry("META-INF/services/org.apache.nifi.reporting.ReportingTask");
-            final JarEntry controllerServiceEntry = jarFile
-                    .getJarEntry("META-INF/services/org.apache.nifi.controller.ControllerService");
+            final JarEntry processorEntry = jarFile.getJarEntry("META-INF/services/org.apache.nifi.processor.Processor");
+            final JarEntry reportingTaskEntry = jarFile.getJarEntry("META-INF/services/org.apache.nifi.reporting.ReportingTask");
+            final JarEntry controllerServiceEntry = jarFile.getJarEntry("META-INF/services/org.apache.nifi.controller.ControllerService");
 
             if (processorEntry==null && reportingTaskEntry==null && controllerServiceEntry==null) {
-                return false;
+                return mapping;
             }
 
-            extensionMapping.addAllProcessors(determineDocumentedNiFiComponents(jarFile,
-                    processorEntry));
-            extensionMapping.addAllReportingTasks(determineDocumentedNiFiComponents(jarFile,
-                    reportingTaskEntry));
-            extensionMapping.addAllControllerServices(determineDocumentedNiFiComponents(jarFile,
-                    controllerServiceEntry));
-            return true;
+            mapping.addAllProcessors(coordinate, determineDocumentedNiFiComponents(jarFile, processorEntry));
+            mapping.addAllReportingTasks(coordinate, determineDocumentedNiFiComponents(jarFile, reportingTaskEntry));
+            mapping.addAllControllerServices(coordinate, determineDocumentedNiFiComponents(jarFile, controllerServiceEntry));
+            return mapping;
         }
     }
 
-    private static List<String> determineDocumentedNiFiComponents(final JarFile jarFile,
-            final JarEntry jarEntry) throws IOException {
+    private static List<String> determineDocumentedNiFiComponents(final JarFile jarFile, final JarEntry jarEntry) throws IOException {
         final List<String> componentNames = new ArrayList<>();
 
         if (jarEntry == null) {
@@ -341,15 +350,14 @@ public final class NarUnpacker {
         }
 
         try (final InputStream entryInputStream = jarFile.getInputStream(jarEntry);
-                final BufferedReader reader = new BufferedReader(new InputStreamReader(
-                        entryInputStream))) {
+                final BufferedReader reader = new BufferedReader(new InputStreamReader(entryInputStream))) {
+
             String line;
             while ((line = reader.readLine()) != null) {
                 final String trimmedLine = line.trim();
                 if (!trimmedLine.isEmpty() && !trimmedLine.startsWith("#")) {
                     final int indexOfPound = trimmedLine.indexOf("#");
-                    final String effectiveLine = (indexOfPound > 0) ? trimmedLine.substring(0,
-                            indexOfPound) : trimmedLine;
+                    final String effectiveLine = (indexOfPound > 0) ? trimmedLine.substring(0, indexOfPound) : trimmedLine;
                     componentNames.add(effectiveLine);
                 }
             }

http://git-wip-us.apache.org/repos/asf/nifi/blob/d90cf846/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/test/java/org/apache/nifi/nar/NarBundleUtilTest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/test/java/org/apache/nifi/nar/NarBundleUtilTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/test/java/org/apache/nifi/nar/NarBundleUtilTest.java
new file mode 100644
index 0000000..aa526c6
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/test/java/org/apache/nifi/nar/NarBundleUtilTest.java
@@ -0,0 +1,101 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.nar;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
+import org.apache.nifi.bundle.BundleCoordinate;
+import org.apache.nifi.bundle.BundleDetails;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+
+public class NarBundleUtilTest {
+
+    @Test
+    public void testManifestWithVersioningAndBuildInfo() throws IOException {
+        final File narDir = new File("src/test/resources/nars/nar-with-versioning");
+        final BundleDetails narDetails = NarBundleUtil.fromNarDirectory(narDir);
+        assertEquals(narDir.getPath(), narDetails.getWorkingDirectory().getPath());
+
+        assertEquals("org.apache.nifi", narDetails.getCoordinate().getGroup());
+        assertEquals("nifi-hadoop-nar", narDetails.getCoordinate().getId());
+        assertEquals("1.2.0", narDetails.getCoordinate().getVersion());
+
+        assertEquals("org.apache.nifi.hadoop", narDetails.getDependencyCoordinate().getGroup());
+        assertEquals("nifi-hadoop-libraries-nar", narDetails.getDependencyCoordinate().getId());
+        assertEquals("1.2.1", narDetails.getDependencyCoordinate().getVersion());
+
+        assertEquals("NIFI-3380", narDetails.getBuildBranch());
+        assertEquals("1.8.0_74", narDetails.getBuildJdk());
+        assertEquals("a032175", narDetails.getBuildRevision());
+        assertEquals("HEAD", narDetails.getBuildTag());
+        assertEquals("2017-01-23T10:36:27Z", narDetails.getBuildTimestamp());
+        assertEquals("bbende", narDetails.getBuiltBy());
+    }
+
+    @Test
+    public void testManifestWithoutVersioningAndBuildInfo() throws IOException {
+        final File narDir = new File("src/test/resources/nars/nar-without-versioning");
+        final BundleDetails narDetails = NarBundleUtil.fromNarDirectory(narDir);
+        assertEquals(narDir.getPath(), narDetails.getWorkingDirectory().getPath());
+
+        assertEquals(BundleCoordinate.DEFAULT_GROUP, narDetails.getCoordinate().getGroup());
+        assertEquals("nifi-hadoop-nar", narDetails.getCoordinate().getId());
+        assertEquals(BundleCoordinate.DEFAULT_VERSION, narDetails.getCoordinate().getVersion());
+
+        assertEquals(BundleCoordinate.DEFAULT_GROUP, narDetails.getDependencyCoordinate().getGroup());
+        assertEquals("nifi-hadoop-libraries-nar", narDetails.getDependencyCoordinate().getId());
+        assertEquals(BundleCoordinate.DEFAULT_VERSION, narDetails.getDependencyCoordinate().getVersion());
+
+        assertNull(narDetails.getBuildBranch());
+        assertEquals("1.8.0_74", narDetails.getBuildJdk());
+        assertNull(narDetails.getBuildRevision());
+        assertNull(narDetails.getBuildTag());
+        assertNull(narDetails.getBuildTimestamp());
+        assertEquals("bbende", narDetails.getBuiltBy());
+    }
+
+    @Test
+    public void testManifestWithoutNarDependency() throws IOException {
+        final File narDir = new File("src/test/resources/nars/nar-without-dependency");
+        final BundleDetails narDetails = NarBundleUtil.fromNarDirectory(narDir);
+        assertEquals(narDir.getPath(), narDetails.getWorkingDirectory().getPath());
+
+        assertEquals("org.apache.nifi", narDetails.getCoordinate().getGroup());
+        assertEquals("nifi-hadoop-nar", narDetails.getCoordinate().getId());
+        assertEquals("1.2.0", narDetails.getCoordinate().getVersion());
+
+        assertNull(narDetails.getDependencyCoordinate());
+
+        assertEquals("NIFI-3380", narDetails.getBuildBranch());
+        assertEquals("1.8.0_74", narDetails.getBuildJdk());
+        assertEquals("a032175", narDetails.getBuildRevision());
+        assertEquals("HEAD", narDetails.getBuildTag());
+        assertEquals("2017-01-23T10:36:27Z", narDetails.getBuildTimestamp());
+        assertEquals("bbende", narDetails.getBuiltBy());
+    }
+
+    @Test(expected = IOException.class)
+    public void testFromManifestWhenNarDirectoryDoesNotExist() throws IOException {
+        final File manifest = new File("src/test/resources/nars/nar-does-not-exist");
+        NarBundleUtil.fromNarDirectory(manifest);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/d90cf846/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/test/java/org/apache/nifi/nar/NarThreadContextClassLoaderTest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/test/java/org/apache/nifi/nar/NarThreadContextClassLoaderTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/test/java/org/apache/nifi/nar/NarThreadContextClassLoaderTest.java
index c44dff2..4528c0a 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/test/java/org/apache/nifi/nar/NarThreadContextClassLoaderTest.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/test/java/org/apache/nifi/nar/NarThreadContextClassLoaderTest.java
@@ -17,10 +17,17 @@
 package org.apache.nifi.nar;
 
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertNotNull;
 
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
 
+import org.apache.nifi.bundle.Bundle;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.util.NiFiProperties;
 import org.junit.Test;
 
@@ -29,8 +36,14 @@ public class NarThreadContextClassLoaderTest {
     @Test
     public void validateWithPropertiesConstructor() throws Exception {
         NiFiProperties properties = NiFiProperties.createBasicNiFiProperties("src/test/resources/nifi.properties", null);
-        assertTrue(NarThreadContextClassLoader.createInstance(WithPropertiesConstructor.class.getName(),
-                WithPropertiesConstructor.class, properties) instanceof WithPropertiesConstructor);
+        Bundle systemBundle = ExtensionManager.createSystemBundle(properties);
+        ExtensionManager.discoverExtensions(systemBundle, Collections.emptySet());
+
+        Object obj = NarThreadContextClassLoader.createInstance(WithPropertiesConstructor.class.getName(),
+                WithPropertiesConstructor.class, properties);
+        assertTrue(obj instanceof WithPropertiesConstructor);
+        WithPropertiesConstructor withPropertiesConstructor = (WithPropertiesConstructor) obj;
+        assertNotNull(withPropertiesConstructor.properties);
     }
 
     @Test(expected = IllegalStateException.class)
@@ -38,39 +51,49 @@ public class NarThreadContextClassLoaderTest {
         Map<String, String> additionalProperties = new HashMap<>();
         additionalProperties.put("fail", "true");
         NiFiProperties properties = NiFiProperties.createBasicNiFiProperties("src/test/resources/nifi.properties", additionalProperties);
+        Bundle systemBundle = ExtensionManager.createSystemBundle(properties);
+        ExtensionManager.discoverExtensions(systemBundle, Collections.emptySet());
         NarThreadContextClassLoader.createInstance(WithPropertiesConstructor.class.getName(), WithPropertiesConstructor.class, properties);
     }
 
     @Test
     public void validateWithDefaultConstructor() throws Exception {
         NiFiProperties properties = NiFiProperties.createBasicNiFiProperties("src/test/resources/nifi.properties", null);
+        Bundle systemBundle = ExtensionManager.createSystemBundle(properties);
+        ExtensionManager.discoverExtensions(systemBundle, Collections.emptySet());
         assertTrue(NarThreadContextClassLoader.createInstance(WithDefaultConstructor.class.getName(),
                 WithDefaultConstructor.class, properties) instanceof WithDefaultConstructor);
     }
 
-    @Test(expected = IllegalStateException.class)
-    public void validateWithWrongConstructor() throws Exception {
-        NiFiProperties properties = NiFiProperties.createBasicNiFiProperties("src/test/resources/nifi.properties", null);
-        NarThreadContextClassLoader.createInstance(WrongConstructor.class.getName(), WrongConstructor.class, properties);
-    }
+    public static class WithPropertiesConstructor extends AbstractProcessor {
+        private NiFiProperties properties;
+
+        public WithPropertiesConstructor() {
+
+        }
 
-    public static class WithPropertiesConstructor {
         public WithPropertiesConstructor(NiFiProperties properties) {
             if (properties.getProperty("fail") != null) {
                 throw new RuntimeException("Intentional failure");
             }
+            this.properties = properties;
+        }
+
+        @Override
+        public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
+
         }
     }
 
-    public static class WithDefaultConstructor {
+    public static class WithDefaultConstructor extends AbstractProcessor {
         public WithDefaultConstructor() {
 
         }
-    }
 
-    public static class WrongConstructor {
-        public WrongConstructor(String s) {
+        @Override
+        public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
 
         }
     }
+
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/d90cf846/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/test/java/org/apache/nifi/nar/NarUnpackerTest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/test/java/org/apache/nifi/nar/NarUnpackerTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/test/java/org/apache/nifi/nar/NarUnpackerTest.java
index 0fc8f4d..8cc2ccd 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/test/java/org/apache/nifi/nar/NarUnpackerTest.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/test/java/org/apache/nifi/nar/NarUnpackerTest.java
@@ -30,13 +30,13 @@ import java.nio.file.Path;
 import java.nio.file.Paths;
 import java.nio.file.SimpleFileVisitor;
 import java.nio.file.attribute.BasicFileAttributes;
+import java.util.Collections;
+import java.util.HashMap;
 import java.util.HashSet;
+import java.util.Map;
 import java.util.Set;
 
 import static java.nio.file.StandardCopyOption.REPLACE_EXISTING;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
@@ -88,14 +88,12 @@ public class NarUnpackerTest {
         assertEquals("./target/NarUnpacker/lib2/",
                 properties.getProperty("nifi.nar.library.directory.alt"));
 
-        final ExtensionMapping extensionMapping = NarUnpacker.unpackNars(properties);
+        final ExtensionMapping extensionMapping = NarUnpacker.unpackNars(properties, ExtensionManager.createSystemBundle(properties));
 
         assertEquals(2, extensionMapping.getAllExtensionNames().size());
 
-        assertTrue(extensionMapping.getAllExtensionNames().contains(
-                "org.apache.nifi.processors.dummy.one"));
-        assertTrue(extensionMapping.getAllExtensionNames().contains(
-                "org.apache.nifi.processors.dummy.two"));
+        assertTrue(extensionMapping.getAllExtensionNames().keySet().contains("org.apache.nifi.processors.dummy.one"));
+        assertTrue(extensionMapping.getAllExtensionNames().keySet().contains("org.apache.nifi.processors.dummy.two"));
         final File extensionsWorkingDir = properties.getExtensionsWorkingDirectory();
         File[] extensionFiles = extensionsWorkingDir.listFiles();
 
@@ -121,11 +119,10 @@ public class NarUnpackerTest {
         others.put("nifi.nar.library.directory.alt", emptyDir.toString());
         NiFiProperties properties = loadSpecifiedProperties("/NarUnpacker/conf/nifi.properties", others);
 
-        final ExtensionMapping extensionMapping = NarUnpacker.unpackNars(properties);
+        final ExtensionMapping extensionMapping = NarUnpacker.unpackNars(properties, ExtensionManager.createSystemBundle(properties));
 
         assertEquals(1, extensionMapping.getAllExtensionNames().size());
-        assertTrue(extensionMapping.getAllExtensionNames().contains(
-                "org.apache.nifi.processors.dummy.one"));
+        assertTrue(extensionMapping.getAllExtensionNames().keySet().contains("org.apache.nifi.processors.dummy.one"));
 
         final File extensionsWorkingDir = properties.getExtensionsWorkingDirectory();
         File[] extensionFiles = extensionsWorkingDir.listFiles();
@@ -145,10 +142,9 @@ public class NarUnpackerTest {
         others.put("nifi.nar.library.directory.alt", nonExistantDir.toString());
         NiFiProperties properties = loadSpecifiedProperties("/NarUnpacker/conf/nifi.properties", others);
 
-        final ExtensionMapping extensionMapping = NarUnpacker.unpackNars(properties);
+        final ExtensionMapping extensionMapping = NarUnpacker.unpackNars(properties, ExtensionManager.createSystemBundle(properties));
 
-        assertTrue(extensionMapping.getAllExtensionNames().contains(
-                "org.apache.nifi.processors.dummy.one"));
+        assertTrue(extensionMapping.getAllExtensionNames().keySet().contains("org.apache.nifi.processors.dummy.one"));
 
         assertEquals(1, extensionMapping.getAllExtensionNames().size());
 
@@ -170,7 +166,7 @@ public class NarUnpackerTest {
         others.put("nifi.nar.library.directory.alt", nonDir.toString());
         NiFiProperties properties = loadSpecifiedProperties("/NarUnpacker/conf/nifi.properties", others);
 
-        final ExtensionMapping extensionMapping = NarUnpacker.unpackNars(properties);
+        final ExtensionMapping extensionMapping = NarUnpacker.unpackNars(properties, ExtensionManager.createSystemBundle(properties));
 
         assertNull(extensionMapping);
     }
@@ -180,8 +176,7 @@ public class NarUnpackerTest {
         try {
             filePath = NarUnpackerTest.class.getResource(propertiesFile).toURI().getPath();
         } catch (URISyntaxException ex) {
-            throw new RuntimeException("Cannot load properties file due to "
-                    + ex.getLocalizedMessage(), ex);
+            throw new RuntimeException("Cannot load properties file due to " + ex.getLocalizedMessage(), ex);
         }
         return NiFiProperties.createBasicNiFiProperties(filePath, others);
     }

http://git-wip-us.apache.org/repos/asf/nifi/blob/d90cf846/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/test/resources/META-INF/services/org.apache.nifi.processor.Processor
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/test/resources/META-INF/services/org.apache.nifi.processor.Processor b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/test/resources/META-INF/services/org.apache.nifi.processor.Processor
new file mode 100644
index 0000000..9d180b6
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/test/resources/META-INF/services/org.apache.nifi.processor.Processor
@@ -0,0 +1,16 @@
+# 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.nar.NarThreadContextClassLoaderTest$WithPropertiesConstructor
+org.apache.nifi.nar.NarThreadContextClassLoaderTest$WithDefaultConstructor

http://git-wip-us.apache.org/repos/asf/nifi/blob/d90cf846/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/test/resources/NarUnpacker/conf/nifi.properties
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/test/resources/NarUnpacker/conf/nifi.properties b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/test/resources/NarUnpacker/conf/nifi.properties
index 69b5b09..0559752 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/test/resources/NarUnpacker/conf/nifi.properties
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/test/resources/NarUnpacker/conf/nifi.properties
@@ -14,7 +14,6 @@
 # limitations under the License.
 
 # Core Properties #
-nifi.version=nifi-test 3.0.0
 nifi.flow.configuration.file=./target/flow.xml.gz
 nifi.flow.configuration.archive.dir=./target/archive/
 nifi.flowcontroller.autoResumeState=true

http://git-wip-us.apache.org/repos/asf/nifi/blob/d90cf846/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/test/resources/nars/nar-with-versioning/META-INF/MANIFEST.MF
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/test/resources/nars/nar-with-versioning/META-INF/MANIFEST.MF b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/test/resources/nars/nar-with-versioning/META-INF/MANIFEST.MF
new file mode 100644
index 0000000..51a51a5
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/test/resources/nars/nar-with-versioning/META-INF/MANIFEST.MF
@@ -0,0 +1,15 @@
+Manifest-Version: 1.0
+Archiver-Version: Plexus Archiver
+Build-Tag: HEAD
+Build-Revision: a032175
+Build-Branch: NIFI-3380
+Build-Timestamp: 2017-01-23T10:36:27Z
+Build-Jdk: 1.8.0_74
+Built-By: bbende
+Nar-Group: org.apache.nifi
+Nar-Id: nifi-hadoop-nar
+Nar-Version: 1.2.0
+Nar-Dependency-Group: org.apache.nifi.hadoop
+Nar-Dependency-Id: nifi-hadoop-libraries-nar
+Nar-Dependency-Version: 1.2.1
+Created-By: Apache Maven 3.3.3

http://git-wip-us.apache.org/repos/asf/nifi/blob/d90cf846/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/test/resources/nars/nar-without-dependency/META-INF/MANIFEST.MF
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/test/resources/nars/nar-without-dependency/META-INF/MANIFEST.MF b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/test/resources/nars/nar-without-dependency/META-INF/MANIFEST.MF
new file mode 100644
index 0000000..de15ce5
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/test/resources/nars/nar-without-dependency/META-INF/MANIFEST.MF
@@ -0,0 +1,12 @@
+Manifest-Version: 1.0
+Archiver-Version: Plexus Archiver
+Build-Tag: HEAD
+Build-Revision: a032175
+Build-Branch: NIFI-3380
+Build-Timestamp: 2017-01-23T10:36:27Z
+Build-Jdk: 1.8.0_74
+Built-By: bbende
+Nar-Group: org.apache.nifi
+Nar-Id: nifi-hadoop-nar
+Nar-Version: 1.2.0
+Created-By: Apache Maven 3.3.3

http://git-wip-us.apache.org/repos/asf/nifi/blob/d90cf846/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/test/resources/nars/nar-without-versioning/META-INF/MANIFEST.MF
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/test/resources/nars/nar-without-versioning/META-INF/MANIFEST.MF b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/test/resources/nars/nar-without-versioning/META-INF/MANIFEST.MF
new file mode 100644
index 0000000..219e61d
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/test/resources/nars/nar-without-versioning/META-INF/MANIFEST.MF
@@ -0,0 +1,7 @@
+Manifest-Version: 1.0
+Archiver-Version: Plexus Archiver
+Build-Jdk: 1.8.0_74
+Built-By: bbende
+Nar-Id: nifi-hadoop-nar
+Nar-Dependency-Id: nifi-hadoop-libraries-nar
+Created-By: Apache Maven 3.3.3

http://git-wip-us.apache.org/repos/asf/nifi/blob/d90cf846/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/test/resources/nifi.properties
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/test/resources/nifi.properties b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/test/resources/nifi.properties
index c2cbad1..a55b398 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/test/resources/nifi.properties
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/test/resources/nifi.properties
@@ -14,7 +14,6 @@
 # limitations under the License.
 
 # Core Properties #
-nifi.version=1.1.0-SNAPSHOT
 nifi.flow.configuration.file=./conf/flow.xml.gz
 nifi.flow.configuration.archive.enabled=true
 nifi.flow.configuration.archive.dir=./conf/archive/

http://git-wip-us.apache.org/repos/asf/nifi/blob/d90cf846/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/groovy/org/apache/nifi/properties/ProtectedNiFiPropertiesGroovyTest.groovy
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/groovy/org/apache/nifi/properties/ProtectedNiFiPropertiesGroovyTest.groovy b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/groovy/org/apache/nifi/properties/ProtectedNiFiPropertiesGroovyTest.groovy
index bf4e677..0d5c976 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/groovy/org/apache/nifi/properties/ProtectedNiFiPropertiesGroovyTest.groovy
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/groovy/org/apache/nifi/properties/ProtectedNiFiPropertiesGroovyTest.groovy
@@ -237,7 +237,7 @@ class ProtectedNiFiPropertiesGroovyTest extends GroovyTestCase {
     @Test
     public void testShouldGetAdditionalSensitiveProperties() throws Exception {
         // Arrange
-        def completeSensitiveProperties = DEFAULT_SENSITIVE_PROPERTIES + ["nifi.ui.banner.text", "nifi.version"]
+        def completeSensitiveProperties = DEFAULT_SENSITIVE_PROPERTIES + ["nifi.ui.banner.text"]
         logger.expected("${completeSensitiveProperties.size()} total sensitive properties: ${completeSensitiveProperties.join(", ")}")
 
         ProtectedNiFiProperties properties = loadFromFile("/conf/nifi_with_additional_sensitive_keys.properties")

http://git-wip-us.apache.org/repos/asf/nifi/blob/d90cf846/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/resources/bootstrap_tests/conf/nifi_with_sensitive_properties_protected_aes.properties
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/resources/bootstrap_tests/conf/nifi_with_sensitive_properties_protected_aes.properties b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/resources/bootstrap_tests/conf/nifi_with_sensitive_properties_protected_aes.properties
index bee200e..2a67d27 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/resources/bootstrap_tests/conf/nifi_with_sensitive_properties_protected_aes.properties
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/resources/bootstrap_tests/conf/nifi_with_sensitive_properties_protected_aes.properties
@@ -14,7 +14,6 @@
 # limitations under the License.
 
 # Core Properties #
-nifi.version=nifi-test 3.0.0
 nifi.flow.configuration.file=./target/flow.xml.gz
 nifi.flow.configuration.archive.dir=./target/archive/
 nifi.flowcontroller.autoResumeState=true

http://git-wip-us.apache.org/repos/asf/nifi/blob/d90cf846/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/resources/conf/nifi.blank.properties
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/resources/conf/nifi.blank.properties b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/resources/conf/nifi.blank.properties
index 898cebf..aaf2e29 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/resources/conf/nifi.blank.properties
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/resources/conf/nifi.blank.properties
@@ -14,7 +14,6 @@
 # limitations under the License.
 
 # Core Properties #
-nifi.version=nifi-test 3.0.0
 nifi.flow.configuration.file=./target/flow.xml.gz
 nifi.flow.configuration.archive.dir=./target/archive/
 nifi.flowcontroller.autoResumeState=true

http://git-wip-us.apache.org/repos/asf/nifi/blob/d90cf846/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/resources/conf/nifi.missing.properties
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/resources/conf/nifi.missing.properties b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/resources/conf/nifi.missing.properties
index 786b05f..fb48be3 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/resources/conf/nifi.missing.properties
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/resources/conf/nifi.missing.properties
@@ -14,7 +14,6 @@
 # limitations under the License.
 
 # Core Properties #
-nifi.version=nifi-test 3.0.0
 nifi.flow.configuration.file=./target/flow.xml.gz
 nifi.flow.configuration.archive.dir=./target/archive/
 nifi.flowcontroller.autoResumeState=true

http://git-wip-us.apache.org/repos/asf/nifi/blob/d90cf846/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/resources/conf/nifi.properties
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/resources/conf/nifi.properties b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/resources/conf/nifi.properties
index f9d9b78..6d1e03b 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/resources/conf/nifi.properties
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/resources/conf/nifi.properties
@@ -14,7 +14,6 @@
 # limitations under the License.
 
 # Core Properties #
-nifi.version=nifi-test 3.0.0
 nifi.flow.configuration.file=./target/flow.xml.gz
 nifi.flow.configuration.archive.dir=./target/archive/
 nifi.flowcontroller.autoResumeState=true

http://git-wip-us.apache.org/repos/asf/nifi/blob/d90cf846/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/resources/conf/nifi_with_additional_sensitive_keys.properties
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/resources/conf/nifi_with_additional_sensitive_keys.properties b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/resources/conf/nifi_with_additional_sensitive_keys.properties
index 85728e6..f775d83 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/resources/conf/nifi_with_additional_sensitive_keys.properties
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/resources/conf/nifi_with_additional_sensitive_keys.properties
@@ -14,7 +14,6 @@
 # limitations under the License.
 
 # Core Properties #
-nifi.version=nifi-test 3.0.0
 nifi.flow.configuration.file=./target/flow.xml.gz
 nifi.flow.configuration.archive.dir=./target/archive/
 nifi.flowcontroller.autoResumeState=true
@@ -74,7 +73,7 @@ nifi.web.jetty.working.directory=./target/work/jetty
 nifi.sensitive.props.key=key
 nifi.sensitive.props.algorithm=PBEWITHMD5AND256BITAES-CBC-OPENSSL
 nifi.sensitive.props.provider=BC
-nifi.sensitive.props.additional.keys=nifi.ui.banner.text, nifi.version
+nifi.sensitive.props.additional.keys=nifi.ui.banner.text
 
 nifi.security.keystore=
 nifi.security.keystoreType=

http://git-wip-us.apache.org/repos/asf/nifi/blob/d90cf846/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/resources/conf/nifi_with_all_sensitive_properties_protected_aes.properties
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/resources/conf/nifi_with_all_sensitive_properties_protected_aes.properties b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/resources/conf/nifi_with_all_sensitive_properties_protected_aes.properties
index bee200e..2a67d27 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/resources/conf/nifi_with_all_sensitive_properties_protected_aes.properties
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/resources/conf/nifi_with_all_sensitive_properties_protected_aes.properties
@@ -14,7 +14,6 @@
 # limitations under the License.
 
 # Core Properties #
-nifi.version=nifi-test 3.0.0
 nifi.flow.configuration.file=./target/flow.xml.gz
 nifi.flow.configuration.archive.dir=./target/archive/
 nifi.flowcontroller.autoResumeState=true

http://git-wip-us.apache.org/repos/asf/nifi/blob/d90cf846/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/resources/conf/nifi_with_recursive_additional_sensitive_keys.properties
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/resources/conf/nifi_with_recursive_additional_sensitive_keys.properties b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/resources/conf/nifi_with_recursive_additional_sensitive_keys.properties
index 1d5009c..6a88c25 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/resources/conf/nifi_with_recursive_additional_sensitive_keys.properties
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/resources/conf/nifi_with_recursive_additional_sensitive_keys.properties
@@ -14,7 +14,6 @@
 # limitations under the License.
 
 # Core Properties #
-nifi.version=nifi-test 3.0.0
 nifi.flow.configuration.file=./target/flow.xml.gz
 nifi.flow.configuration.archive.dir=./target/archive/
 nifi.flowcontroller.autoResumeState=true

http://git-wip-us.apache.org/repos/asf/nifi/blob/d90cf846/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/resources/conf/nifi_with_sensitive_properties_protected_aes.properties
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/resources/conf/nifi_with_sensitive_properties_protected_aes.properties b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/resources/conf/nifi_with_sensitive_properties_protected_aes.properties
index f002cd3..10e91ea 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/resources/conf/nifi_with_sensitive_properties_protected_aes.properties
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/resources/conf/nifi_with_sensitive_properties_protected_aes.properties
@@ -14,7 +14,6 @@
 # limitations under the License.
 
 # Core Properties #
-nifi.version=nifi-test 3.0.0
 nifi.flow.configuration.file=./target/flow.xml.gz
 nifi.flow.configuration.archive.dir=./target/archive/
 nifi.flowcontroller.autoResumeState=true

http://git-wip-us.apache.org/repos/asf/nifi/blob/d90cf846/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/resources/conf/nifi_with_sensitive_properties_protected_aes_128.properties
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/resources/conf/nifi_with_sensitive_properties_protected_aes_128.properties b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/resources/conf/nifi_with_sensitive_properties_protected_aes_128.properties
index 1da81f8..c45ab86 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/resources/conf/nifi_with_sensitive_properties_protected_aes_128.properties
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/resources/conf/nifi_with_sensitive_properties_protected_aes_128.properties
@@ -14,7 +14,6 @@
 # limitations under the License.
 
 # Core Properties #
-nifi.version=nifi-test 3.0.0
 nifi.flow.configuration.file=./target/flow.xml.gz
 nifi.flow.configuration.archive.dir=./target/archive/
 nifi.flowcontroller.autoResumeState=true
@@ -75,7 +74,7 @@ nifi.sensitive.props.key=6WUpex+VZiN05LXu||joWJMuoSzYniEC7IAoingTimlG7+RGk8I2irl
 nifi.sensitive.props.key.protected=aes/gcm/128
 nifi.sensitive.props.algorithm=PBEWITHMD5AND256BITAES-CBC-OPENSSL
 nifi.sensitive.props.provider=BC
-nifi.sensitive.props.additional.keys=nifi.ui.banner.text, nifi.version
+nifi.sensitive.props.additional.keys=nifi.ui.banner.text
 
 nifi.security.keystore=/path/to/keystore.jks
 nifi.security.keystoreType=JKS

http://git-wip-us.apache.org/repos/asf/nifi/blob/d90cf846/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/resources/conf/nifi_with_sensitive_properties_protected_aes_128_password.properties
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/resources/conf/nifi_with_sensitive_properties_protected_aes_128_password.properties b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/resources/conf/nifi_with_sensitive_properties_protected_aes_128_password.properties
index 9cf5987..51b1364 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/resources/conf/nifi_with_sensitive_properties_protected_aes_128_password.properties
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/resources/conf/nifi_with_sensitive_properties_protected_aes_128_password.properties
@@ -14,7 +14,6 @@
 # limitations under the License.
 
 # Core Properties #
-nifi.version=nifi-test 3.0.0
 nifi.flow.configuration.file=./target/flow.xml.gz
 nifi.flow.configuration.archive.dir=./target/archive/
 nifi.flowcontroller.autoResumeState=true

http://git-wip-us.apache.org/repos/asf/nifi/blob/d90cf846/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/resources/conf/nifi_with_sensitive_properties_protected_aes_multiple_malformed.properties
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/resources/conf/nifi_with_sensitive_properties_protected_aes_multiple_malformed.properties b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/resources/conf/nifi_with_sensitive_properties_protected_aes_multiple_malformed.properties
index 51c880e..cc2153d 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/resources/conf/nifi_with_sensitive_properties_protected_aes_multiple_malformed.properties
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/resources/conf/nifi_with_sensitive_properties_protected_aes_multiple_malformed.properties
@@ -14,7 +14,6 @@
 # limitations under the License.
 
 # Core Properties #
-nifi.version=nifi-test 3.0.0
 nifi.flow.configuration.file=./target/flow.xml.gz
 nifi.flow.configuration.archive.dir=./target/archive/
 nifi.flowcontroller.autoResumeState=true
@@ -75,7 +74,7 @@ nifi.sensitive.props.key=n2z+tTTbHuZ4V4V2||uWhdasyDXD4ZG2lMAes/vqh6u4vaz4xgL4aEb
 nifi.sensitive.props.key.protected=aes/gcm/256
 nifi.sensitive.props.algorithm=PBEWITHMD5AND256BITAES-CBC-OPENSSL
 nifi.sensitive.props.provider=BC
-nifi.sensitive.props.additional.keys=nifi.ui.banner.text, nifi.version
+nifi.sensitive.props.additional.keys=nifi.ui.banner.text
 
 nifi.security.keystore=/path/to/keystore.jks
 nifi.security.keystoreType=JKS

http://git-wip-us.apache.org/repos/asf/nifi/blob/d90cf846/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/resources/conf/nifi_with_sensitive_properties_protected_aes_single_malformed.properties
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/resources/conf/nifi_with_sensitive_properties_protected_aes_single_malformed.properties b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/resources/conf/nifi_with_sensitive_properties_protected_aes_single_malformed.properties
index 6078bf7..03fe7f3 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/resources/conf/nifi_with_sensitive_properties_protected_aes_single_malformed.properties
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/resources/conf/nifi_with_sensitive_properties_protected_aes_single_malformed.properties
@@ -14,7 +14,6 @@
 # limitations under the License.
 
 # Core Properties #
-nifi.version=nifi-test 3.0.0
 nifi.flow.configuration.file=./target/flow.xml.gz
 nifi.flow.configuration.archive.dir=./target/archive/
 nifi.flowcontroller.autoResumeState=true

http://git-wip-us.apache.org/repos/asf/nifi/blob/d90cf846/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/resources/conf/nifi_with_sensitive_properties_protected_unknown.properties
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/resources/conf/nifi_with_sensitive_properties_protected_unknown.properties b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/resources/conf/nifi_with_sensitive_properties_protected_unknown.properties
index 07c5ba7..591dfaa 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/resources/conf/nifi_with_sensitive_properties_protected_unknown.properties
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/resources/conf/nifi_with_sensitive_properties_protected_unknown.properties
@@ -14,7 +14,6 @@
 # limitations under the License.
 
 # Core Properties #
-nifi.version=nifi-test 3.0.0
 nifi.flow.configuration.file=./target/flow.xml.gz
 nifi.flow.configuration.archive.dir=./target/archive/
 nifi.flowcontroller.autoResumeState=true

http://git-wip-us.apache.org/repos/asf/nifi/blob/d90cf846/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/resources/conf/nifi_with_sensitive_properties_unprotected.properties
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/resources/conf/nifi_with_sensitive_properties_unprotected.properties b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/resources/conf/nifi_with_sensitive_properties_unprotected.properties
index b81c369..5ff84a8 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/resources/conf/nifi_with_sensitive_properties_unprotected.properties
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/resources/conf/nifi_with_sensitive_properties_unprotected.properties
@@ -14,7 +14,6 @@
 # limitations under the License.
 
 # Core Properties #
-nifi.version=nifi-test 3.0.0
 nifi.flow.configuration.file=./target/flow.xml.gz
 nifi.flow.configuration.archive.dir=./target/archive/
 nifi.flowcontroller.autoResumeState=true